You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bo...@apache.org on 2018/05/14 19:44:22 UTC

[geode] 01/01: GEODE-5145: Added support for destroying an index during region creation

This is an automated email from the ASF dual-hosted git repository.

boglesby pushed a commit to branch feature/GEODE-5145
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 0326713f5a557820cd36c7325d6540e804893b0f
Author: Barry Oglesby <bo...@pivotal.io>
AuthorDate: Wed May 2 09:43:33 2018 -0700

    GEODE-5145: Added support for destroying an index during region creation
---
 .../apache/geode/internal/cache/LocalRegion.java   |   4 +
 .../apache/geode/management/cli/GfshCommand.java   |   8 +
 .../geode/management/internal/cli/CliUtil.java     |  27 ++
 .../lucene/LuceneIndexDestroyedException.java      |   2 +-
 .../lucene/internal/DestroyLuceneIndexMessage.java |  21 +-
 .../internal/LuceneIndexCreationProfile.java       |   4 +
 .../internal/LuceneIndexForPartitionedRegion.java  |  31 +-
 .../cache/lucene/internal/LuceneIndexImpl.java     |   4 +
 .../lucene/internal/LuceneRegionListener.java      |   8 +-
 .../cache/lucene/internal/LuceneServiceImpl.java   |  31 +-
 .../lucene/internal/cli/LuceneCliStrings.java      |   2 +
 .../lucene/internal/cli/LuceneIndexCommands.java   |  84 ++---
 .../cli/functions/LuceneDestroyIndexFunction.java  |  31 +-
 .../cache/lucene/LuceneIndexDestroyDUnitTest.java  |  45 +++
 .../cli/DestroyLuceneIndexCommandsDUnitTest.java   | 407 +++++++++++++++++++++
 .../internal/cli/LuceneIndexCommandsDUnitTest.java |   8 +
 .../internal/cli/LuceneIndexCommandsJUnitTest.java | 102 ++----
 .../LuceneDestroyIndexFunctionJUnitTest.java       |   3 +-
 18 files changed, 669 insertions(+), 153 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index ec65991..626f9e8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -10497,6 +10497,10 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     this.cacheServiceProfiles.put(profile.getId(), profile);
   }
 
+  public void removeCacheServiceProfile(String profileId) {
+    this.cacheServiceProfiles.remove(profileId);
+  }
+
   @Override
   public LoaderHelper createLoaderHelper(Object key, Object callbackArgument,
       boolean netSearchAllowed, boolean netLoadAllowed, SearchLoadAndWriteProcessor searcher) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java b/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java
index ab81ab2..1841eac 100644
--- a/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java
@@ -30,6 +30,7 @@ import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.ConfigurationPersistenceService;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundException;
@@ -124,6 +125,13 @@ public abstract class GfshCommand implements CommandMarker {
     return CliUtil.getAllNormalMembers(cache);
   }
 
+  /**
+   * Get All members >= a specific version, excluding locators
+   */
+  public Set<DistributedMember> getNormalMembersWithSameOrNewerVersion(Version version) {
+    return CliUtil.getNormalMembersWithSameOrNewerVersion(cache, version);
+  }
+
   public Execution getMembersFunctionExecutor(final Set<DistributedMember> members) {
     return FunctionService.onMembers(members);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
index 715bab1..8634145 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliUtil.java
@@ -49,7 +49,9 @@ import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.internal.lang.StringUtils;
@@ -184,6 +186,31 @@ public class CliUtil {
   }
 
   /**
+   * Returns a set of all the members of the distributed system of a specific version excluding
+   * locators.
+   */
+  @SuppressWarnings("unchecked")
+  public static Set<DistributedMember> getNormalMembersWithSameOrNewerVersion(InternalCache cache,
+      Version version) {
+    return getAllNormalMembers(cache).stream().filter(
+        member -> ((InternalDistributedMember) member).getVersionObject().compareTo(version) >= 0)
+        .collect(Collectors.toSet());
+  }
+
+  /**
+   * Returns a set of all the members of the distributed system older than a specific version
+   * excluding locators.
+   */
+  @SuppressWarnings("unchecked")
+  public static Set<DistributedMember> getNormalMembersWithSameOrOlderVersion(InternalCache cache,
+      Version version) {
+    return getAllNormalMembers(cache).stream()
+        .filter(member -> ((InternalDistributedMember) member).getVersionObject().equals(version))
+        .collect(Collectors.toSet());
+    // dm.removeMembersWithSameOrNewerVersion(oldMembers, version);
+  }
+
+  /**
    * Returns a set of all the members of the distributed system including locators.
    */
   @SuppressWarnings("unchecked")
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexDestroyedException.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexDestroyedException.java
index 7b38f24..a0365e6 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexDestroyedException.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexDestroyedException.java
@@ -27,7 +27,7 @@ public class LuceneIndexDestroyedException extends GemFireException {
   private final String regionPath;
 
   public LuceneIndexDestroyedException(String indexName, String regionPath) {
-    super();
+    super("Lucene index " + indexName + " on region " + regionPath + " has been destroyed");
     this.indexName = indexName;
     this.regionPath = regionPath;
   }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/DestroyLuceneIndexMessage.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/DestroyLuceneIndexMessage.java
index abf7336..93c620d 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/DestroyLuceneIndexMessage.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/DestroyLuceneIndexMessage.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
+import org.apache.geode.cache.Region;
 import org.apache.geode.cache.lucene.LuceneServiceProvider;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.MessageWithReply;
@@ -29,6 +30,7 @@ import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.distributed.internal.ReplyException;
 import org.apache.geode.distributed.internal.ReplyMessage;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 
 public class DestroyLuceneIndexMessage extends PooledDistributionMessage
@@ -67,10 +69,21 @@ public class DestroyLuceneIndexMessage extends PooledDistributionMessage
       try {
         InternalCache cache = dm.getCache();
         LuceneServiceImpl impl = (LuceneServiceImpl) LuceneServiceProvider.get(cache);
-        impl.destroyIndex(this.indexName, this.regionPath, false);
-        if (logger.isDebugEnabled()) {
-          logger.debug("DestroyLuceneIndexMessage: Destroyed regionPath=" + this.regionPath
-              + "; indexName=" + this.indexName);
+        try {
+          impl.destroyIndex(this.indexName, this.regionPath, false);
+          if (logger.isDebugEnabled()) {
+            logger.debug("DestroyLuceneIndexMessage: Destroyed regionPath=" + this.regionPath
+                + "; indexName=" + this.indexName);
+          }
+        } catch (IllegalArgumentException e) {
+          // If the IllegalArgumentException is index not found, then its ok; otherwise rethrow it.
+          String fullRegionPath =
+              regionPath.startsWith(Region.SEPARATOR) ? regionPath : Region.SEPARATOR + regionPath;
+          String indexNotFoundMessage = LocalizedStrings.LuceneService_INDEX_0_NOT_FOUND_IN_REGION_1
+              .toLocalizedString(this.indexName, fullRegionPath);
+          if (!e.getLocalizedMessage().equals(indexNotFoundMessage)) {
+            throw e;
+          }
         }
       } catch (Throwable e) {
         replyException = new ReplyException(e);
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java
index 3d9b00d..c3b08b3 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexCreationProfile.java
@@ -96,6 +96,10 @@ public class LuceneIndexCreationProfile implements CacheServiceProfile, Versione
 
   @Override
   public String getId() {
+    return generateId(indexName, regionPath);
+  }
+
+  public static String generateId(String indexName, String regionPath) {
     return "lucene_" + LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
   }
 
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index 778b8fd..2343a3c 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -26,6 +26,7 @@ import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.PartitionResolver;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.RegionDestroyedException;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
@@ -44,6 +45,7 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.internal.cache.BucketRegion;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.i18n.LocalizedStrings;
 
 public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   protected Region fileAndChunkRegion;
@@ -208,9 +210,15 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     // localDestroyRegion can't be used because locally destroying regions is not supported on
     // colocated regions
     if (initiator) {
-      fileAndChunkRegion.destroyRegion();
-      if (logger.isDebugEnabled()) {
-        logger.debug("Destroyed fileAndChunkRegion=" + fileAndChunkRegion.getName());
+      try {
+        fileAndChunkRegion.destroyRegion();
+        if (logger.isDebugEnabled()) {
+          logger.debug("Destroyed fileAndChunkRegion=" + fileAndChunkRegion.getName());
+        }
+      } catch (RegionDestroyedException e) {
+        if (logger.isDebugEnabled()) {
+          logger.debug("Already destroyed fileAndChunkRegion=" + fileAndChunkRegion.getName());
+        }
       }
     }
 
@@ -228,9 +236,8 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   private void destroyOnRemoteMembers() {
-    PartitionedRegion pr = (PartitionedRegion) getDataRegion();
-    DistributionManager dm = pr.getDistributionManager();
-    Set<InternalDistributedMember> recipients = pr.getRegionAdvisor().adviseAllPRNodes();
+    DistributionManager dm = getDataRegion().getDistributionManager();
+    Set<InternalDistributedMember> recipients = dm.getOtherNormalDistributionManagerIds();
     if (!recipients.isEmpty()) {
       if (logger.isDebugEnabled()) {
         logger.debug("LuceneIndexForPartitionedRegion: About to send destroy message recipients="
@@ -246,7 +253,17 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       try {
         processor.waitForReplies();
       } catch (ReplyException e) {
-        if (!(e.getCause() instanceof CancelException)) {
+        Throwable cause = e.getCause();
+        if (cause instanceof IllegalArgumentException) {
+          // If the IllegalArgumentException is index not found, then its ok; otherwise rethrow it.
+          String fullRegionPath =
+              regionPath.startsWith(Region.SEPARATOR) ? regionPath : Region.SEPARATOR + regionPath;
+          String indexNotFoundMessage = LocalizedStrings.LuceneService_INDEX_0_NOT_FOUND_IN_REGION_1
+              .toLocalizedString(indexName, fullRegionPath);
+          if (!cause.getLocalizedMessage().equals(indexNotFoundMessage)) {
+            throw e;
+          }
+        } else if (!(cause instanceof CancelException)) {
           throw e;
         }
       } catch (InterruptedException e) {
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
index e58c21f..f06fd91 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
@@ -226,6 +226,10 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
       getDataRegion().getExtensionPoint().removeExtension(extensionToDelete);
     }
 
+    // Remove cache service profile
+    dataRegion
+        .removeCacheServiceProfile(LuceneIndexCreationProfile.generateId(indexName, regionPath));
+
     // Destroy the async event queue
     destroyAsyncEventQueue(initiator);
 
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
index 88d7de0..7977670 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
@@ -22,6 +22,7 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.lucene.LuceneIndexDestroyedException;
 import org.apache.geode.cache.lucene.LuceneSerializer;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalRegionArguments;
@@ -107,7 +108,12 @@ public class LuceneRegionListener implements RegionListener {
   public void afterCreate(Region region) {
     if (region.getFullPath().equals(this.regionPath)
         && this.afterCreateInvoked.compareAndSet(false, true)) {
-      this.service.afterDataRegionCreated(this.luceneIndex);
+      try {
+        this.service.afterDataRegionCreated(this.luceneIndex);
+      } catch (LuceneIndexDestroyedException e) {
+        // @todo log a warning here?
+        return;
+      }
       this.service.createLuceneIndexOnDataRegion((PartitionedRegion) region, luceneIndex);
     }
   }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
index 5267e30..ccb3d06 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
@@ -20,11 +20,11 @@ import static org.apache.geode.internal.DataSerializableFixedID.CREATE_REGION_ME
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
@@ -46,6 +46,7 @@ import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.lucene.LuceneIndex;
+import org.apache.geode.cache.lucene.LuceneIndexDestroyedException;
 import org.apache.geode.cache.lucene.LuceneIndexExistsException;
 import org.apache.geode.cache.lucene.LuceneQueryFactory;
 import org.apache.geode.cache.lucene.LuceneSerializer;
@@ -94,8 +95,8 @@ public class LuceneServiceImpl implements InternalLuceneService {
   private static final Logger logger = LogService.getLogger();
 
   private InternalCache cache;
-  private final HashMap<String, LuceneIndex> indexMap = new HashMap<String, LuceneIndex>();
-  private final HashMap<String, LuceneIndexCreationProfile> definedIndexMap = new HashMap<>();
+  private final Map<String, LuceneIndex> indexMap = new ConcurrentHashMap<>();
+  private final Map<String, LuceneIndexCreationProfile> definedIndexMap = new ConcurrentHashMap<>();
   private IndexListener managementListener;
   public static boolean LUCENE_REINDEX =
       Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "luceneReindex");
@@ -265,7 +266,12 @@ public class LuceneServiceImpl implements InternalLuceneService {
     LuceneIndexImpl luceneIndex = beforeDataRegionCreated(indexName, regionPath,
         region.getAttributes(), analyzer, fieldAnalyzers, aeqId, serializer, fields);
 
-    afterDataRegionCreated(luceneIndex);
+    try {
+      afterDataRegionCreated(luceneIndex);
+    } catch (LuceneIndexDestroyedException e) {
+      // @todo log a warning here?
+      return;
+    }
 
     createLuceneIndexOnDataRegion(region, luceneIndex);
   }
@@ -557,12 +563,21 @@ public class LuceneServiceImpl implements InternalLuceneService {
     // nothing to do there.
   }
 
-  public void registerIndex(LuceneIndex index) {
+  private boolean hasIndexBeenDestroyed(String uniqueIndexName) {
+    return !definedIndexMap.containsKey(uniqueIndexName);
+  }
+
+  private void registerIndex(LuceneIndex index) {
     String regionAndIndex = getUniqueIndexName(index.getName(), index.getRegionPath());
-    if (!indexMap.containsKey(regionAndIndex)) {
-      indexMap.put(regionAndIndex, index);
+    if (hasIndexBeenDestroyed(regionAndIndex)) {
+      ((InternalLuceneIndex) index).destroy(true);
+      throw new LuceneIndexDestroyedException(index.getName(), index.getRegionPath());
+    } else {
+      if (!indexMap.containsKey(regionAndIndex)) {
+        indexMap.put(regionAndIndex, index);
+      }
+      definedIndexMap.remove(regionAndIndex);
     }
-    definedIndexMap.remove(regionAndIndex);
   }
 
   public void unregisterIndex(final String region) {
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneCliStrings.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneCliStrings.java
index 81b6ecc..67d72a9 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneCliStrings.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneCliStrings.java
@@ -97,6 +97,8 @@ public class LuceneCliStrings {
       "Index cannot be empty.";
   public static final String LUCENE_DESTROY_INDEX__MSG__COULDNOT_FIND_MEMBERS_FOR_REGION_0 =
       "Could not find any members defining region {0}.";
+  public static final String LUCENE_DESTROY_INDEX__MSG__COULD_NOT_FIND__MEMBERS_GREATER_THAN_VERSION_0 =
+      "Could not find any members greater than or equal to version {0}.";
   public static final String LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0 =
       "Successfully destroyed all lucene indexes from region {0}";
   public static final String LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FROM_REGION_1 =
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
index 43b7d2c..064c42a 100755
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
@@ -14,10 +14,8 @@
  */
 package org.apache.geode.cache.lucene.internal.cli;
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Objects;
@@ -40,6 +38,7 @@ import org.apache.geode.cache.lucene.internal.cli.functions.LuceneSearchIndexFun
 import org.apache.geode.cache.lucene.internal.security.LucenePermission;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
+import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.management.cli.CliMetaData;
@@ -288,11 +287,27 @@ public class LuceneIndexCommands extends InternalGfshCommand {
     }
 
     authorize(Resource.CLUSTER, Operation.MANAGE, LucenePermission.TARGET);
-    Result result;
-    List<CliFunctionResult> accumulatedResults = new ArrayList<>();
-    final XmlEntity xmlEntity =
-        executeDestroyIndexFunction(accumulatedResults, indexName, regionPath);
-    result = getDestroyIndexResult(accumulatedResults, indexName, regionPath);
+
+    // Get members >= 1.6
+    Set<DistributedMember> validVersionMembers =
+        getNormalMembersWithSameOrNewerVersion(Version.GEODE_160);
+    if (validVersionMembers.isEmpty()) {
+      return ResultBuilder.createInfoResult(CliStrings.format(
+          LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__COULD_NOT_FIND__MEMBERS_GREATER_THAN_VERSION_0,
+          Version.GEODE_160));
+    }
+
+    // Execute the destroy index function
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo(indexName, regionPath);
+    ResultCollector<?, ?> rc =
+        executeFunction(destroyIndexFunction, indexInfo, validVersionMembers);
+
+    // Get the result
+    List<CliFunctionResult> cliFunctionResults = (List<CliFunctionResult>) rc.getResult();
+    Result result = getDestroyIndexResult(cliFunctionResults, indexName, regionPath);
+
+    // Get and process the xml entity
+    XmlEntity xmlEntity = findXmlEntity(cliFunctionResults);
     if (xmlEntity != null) {
       persistClusterConfiguration(result, () -> {
         // Delete the xml entity to remove the index(es) in all groups
@@ -304,61 +319,6 @@ public class LuceneIndexCommands extends InternalGfshCommand {
     return result;
   }
 
-  private XmlEntity executeDestroyIndexFunction(List<CliFunctionResult> accumulatedResults,
-      String indexName, String regionPath) {
-    // Destroy has three cases:
-    //
-    // - no members define the region
-    // In this case, send the request to all members to handle the case where the index has been
-    // created, but not the region
-    //
-    // - all members define the region
-    // In this case, send the request to one of the region members to destroy the index on all
-    // member
-    //
-    // - some members define the region; some don't
-    // In this case, send the request to one of the region members to destroy the index in all the
-    // region members. Then send the function to the remaining members to handle the case where
-    // the index has been created, but not the region
-    XmlEntity xmlEntity = null;
-    Set<DistributedMember> regionMembers = findMembersForRegion(regionPath);
-    Set<DistributedMember> normalMembers = getAllNormalMembers();
-    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo(indexName, regionPath);
-    ResultCollector<?, ?> rc;
-    if (regionMembers.isEmpty()) {
-      // Attempt to destroy the proxy index on all members
-      indexInfo.setDefinedDestroyOnly(true);
-      rc = executeFunction(destroyIndexFunction, indexInfo, normalMembers);
-      accumulatedResults.addAll((List<CliFunctionResult>) rc.getResult());
-    } else {
-      // Attempt to destroy the index on a region member
-      indexInfo.setDefinedDestroyOnly(false);
-      Set<DistributedMember> singleMember = new HashSet<>();
-      singleMember.add(regionMembers.iterator().next());
-      rc = executeFunction(destroyIndexFunction, indexInfo, singleMember);
-      List<CliFunctionResult> cliFunctionResults = (List<CliFunctionResult>) rc.getResult();
-      CliFunctionResult cliFunctionResult = cliFunctionResults.get(0);
-      xmlEntity = cliFunctionResult.getXmlEntity();
-      for (DistributedMember regionMember : regionMembers) {
-        accumulatedResults.add(new CliFunctionResult(regionMember.getId(),
-            cliFunctionResult.isSuccessful(), cliFunctionResult.getMessage()));
-      }
-      // If that succeeds, destroy the proxy index(es) on all other members if necessary
-      if (cliFunctionResult.isSuccessful()) {
-        normalMembers.removeAll(regionMembers);
-        if (!normalMembers.isEmpty()) {
-          indexInfo.setDefinedDestroyOnly(true);
-          rc = executeFunction(destroyIndexFunction, indexInfo, normalMembers);
-          accumulatedResults.addAll((List<CliFunctionResult>) rc.getResult());
-        }
-      } else {
-        // @todo Should dummy results be added to the accumulatedResults for the non-region
-        // members in the failed case
-      }
-    }
-    return xmlEntity;
-  }
-
   private Result getDestroyIndexResult(List<CliFunctionResult> cliFunctionResults, String indexName,
       String regionPath) {
     final TabularResultData tabularResult = ResultBuilder.createTabularResultData();
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunction.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunction.java
index 0583b52..3271556 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunction.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunction.java
@@ -41,8 +41,35 @@ public class LuceneDestroyIndexFunction implements InternalFunction {
           ((LuceneServiceImpl) service).destroyDefinedIndexes(regionPath);
           result = new CliFunctionResult(memberId);
         } else {
-          service.destroyIndexes(regionPath);
-          result = new CliFunctionResult(memberId, getXmlEntity(indexName, regionPath));
+          // Destroy all created indexes
+          CliFunctionResult destroyIndexesResult = null;
+          Exception destroyIndexesException = null;
+          try {
+            service.destroyIndexes(regionPath);
+            destroyIndexesResult =
+                new CliFunctionResult(memberId, getXmlEntity(indexName, regionPath));
+          } catch (Exception e) {
+            destroyIndexesException = e;
+          }
+
+          // Destroy all defined indexes
+          CliFunctionResult destroyDefinedIndexesResult = null;
+          Exception destroyDefinedIndexesException = null;
+          try {
+            ((LuceneServiceImpl) service).destroyDefinedIndexes(regionPath);
+            destroyDefinedIndexesResult = new CliFunctionResult(memberId);
+          } catch (Exception e) {
+            destroyDefinedIndexesException = e;
+          }
+
+          // If there are two exceptions, throw one of them. Note: They should be the same 'No
+          // Lucene indexes were found' exception. Otherwise return the appropriate result.
+          if (destroyIndexesException != null && destroyDefinedIndexesException != null) {
+            throw destroyIndexesException;
+          } else {
+            result =
+                destroyIndexesResult == null ? destroyDefinedIndexesResult : destroyIndexesResult;
+          }
         }
       } else {
         if (indexInfo.isDefinedDestroyOnly()) {
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
index 83541a0..b0a27ea 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
@@ -141,6 +141,43 @@ public class LuceneIndexDestroyDUnitTest extends LuceneDUnitTest {
   }
 
   @Test
+  @Parameters(method = "parametersForIndexDestroys")
+  public void verifyDestroySingleIndexWithDefinedIndexes(boolean destroyDataRegion,
+      RegionTestableType regionType) {
+    // Create index in both members
+    dataStore1.invoke(createIndex());
+    dataStore2.invoke(createIndex());
+
+    // Verify index defined
+    dataStore1.invoke(() -> verifyDefinedIndexCreated());
+    dataStore2.invoke(() -> verifyDefinedIndexCreated());
+
+    // Create region in one member
+    dataStore1.invoke(() -> initDataStore(regionType));
+
+    // Verify index created in one member and defined in the other
+    dataStore1.invoke(() -> verifyIndexCreated());
+    dataStore2.invoke(() -> verifyDefinedIndexCreated());
+
+    // Attempt to destroy data region (should fail)
+    if (destroyDataRegion) {
+      dataStore1.invoke(() -> destroyDataRegion(false, INDEX_NAME));
+    }
+
+    // Destroy index (only needs to be done on one member)
+    dataStore1.invoke(() -> destroyIndex());
+
+    // Verify index destroyed in one member and defined index destroyed in the other
+    dataStore1.invoke(() -> verifyIndexDestroyed());
+    dataStore2.invoke(() -> verifyDefinedIndexDestroyed());
+
+    // Attempt to destroy data region (should succeed)
+    if (destroyDataRegion) {
+      dataStore1.invoke(() -> destroyDataRegion(true));
+    }
+  }
+
+  @Test
   @Parameters(method = "getListOfRegionTestTypes")
   public void verifyDestroySingleIndexWhileDoingPuts(RegionTestableType regionType)
       throws Exception {
@@ -517,12 +554,20 @@ public class LuceneIndexDestroyDUnitTest extends LuceneDUnitTest {
     assertNotNull(luceneService.getIndex(INDEX2_NAME, REGION_NAME));
   }
 
+  private void verifyDefinedIndexCreated() {
+    verifyDefinedIndexCreated(INDEX_NAME, REGION_NAME);
+  }
+
   private void verifyDefinedIndexCreated(String indexName, String regionName) {
     LuceneServiceImpl luceneService = (LuceneServiceImpl) LuceneServiceProvider.get(getCache());
     assertNotNull(luceneService.getDefinedIndex(indexName, regionName));
     assertEquals(1, getCache().getRegionListeners().size());
   }
 
+  private void verifyDefinedIndexDestroyed() {
+    verifyDefinedIndexDestroyed(INDEX_NAME, REGION_NAME);
+  }
+
   private void verifyDefinedIndexDestroyed(String indexName, String regionName) {
     LuceneServiceImpl luceneService = (LuceneServiceImpl) LuceneServiceProvider.get(getCache());
     assertNull(luceneService.getDefinedIndex(indexName, regionName));
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/DestroyLuceneIndexCommandsDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/DestroyLuceneIndexCommandsDUnitTest.java
new file mode 100644
index 0000000..4380c26
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/DestroyLuceneIndexCommandsDUnitTest.java
@@ -0,0 +1,407 @@
+/*
+ * 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.cache.lucene.internal.cli;
+
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+import java.io.Serializable;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.mockito.stubbing.Answer;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.lucene.LuceneService;
+import org.apache.geode.cache.lucene.LuceneServiceProvider;
+import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.assertions.CommandResultAssert;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.LuceneTest;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+
+@Category({DistributedTest.class, LuceneTest.class})
+@RunWith(JUnitParamsRunner.class)
+public class DestroyLuceneIndexCommandsDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public ClusterStartupRule cluster = new ClusterStartupRule();
+
+  private MemberVM locator;
+
+  private MemberVM server1;
+
+  private MemberVM server2;
+
+  private static CountDownLatch indexCreationInProgress;
+
+  private static CountDownLatch indexDestroyComplete;
+
+  @Before
+  public void before() throws Exception {
+    Properties props = new Properties();
+    props.setProperty(SERIALIZABLE_OBJECT_FILTER,
+        "org.apache.geode.cache.lucene.internal.cli.DestroyLuceneIndexCommandsDUnitTest");
+    locator = cluster.startLocatorVM(0, props);
+    server1 = cluster.startServerVM(1, props, locator.getPort());
+    server2 = cluster.startServerVM(2, props, locator.getPort());
+    gfsh.connectAndVerify(locator);
+  }
+
+  @Test
+  @Parameters({"true", "false"})
+  public void testDestroyIndex(boolean createRegion) throws Exception {
+    // Create index and region if necessary
+    server1.invoke(() -> createIndex(1));
+    server2.invoke(() -> createIndex(1));
+    if (createRegion) {
+      server1.invoke(() -> createRegion());
+      server2.invoke(() -> createRegion());
+    }
+
+    // Execute command to destroy index
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --name=index0 --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FROM_REGION_1,
+        new Object[] {"index0", "/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  public void testDestroyIndexWithRegionInOneMember() throws Exception {
+    // Create index in both members
+    server1.invoke(() -> createIndex(1));
+    server2.invoke(() -> createIndex(1));
+
+    // Create region in one member
+    server1.invoke(() -> createRegion());
+
+    // Execute command to destroy index
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --name=index0 --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FROM_REGION_1,
+        new Object[] {"index0", "/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  public void testDestroyNonExistentIndex() throws Exception {
+    // Execute command to destroy index
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --name=index0 --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = LocalizedStrings.LuceneService_INDEX_0_NOT_FOUND_IN_REGION_1
+        .toLocalizedString("index0", "/region");
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  public void testDestroyIndexWithRegionCreationInProgress() throws Exception {
+    server1.invoke(() -> initializeCountDownLatches());
+    server2.invoke(() -> initializeCountDownLatches());
+
+    server1.invoke(() -> createIndexesOnSpy(1));
+    server2.invoke(() -> createIndexesOnSpy(1));
+
+    // Asynchronously create region. This will cause the invokeBeforeAfterDataRegionCreated Answer
+    // to be invoked which will wait for the index to be destroyed before invoking the real
+    // afterDataRegionCreated method and completing region creation. The registerIndex method will
+    // realize the defined index has been destroyed and destroy the real one.
+    AsyncInvocation server1RegionCreationInvocation = server1.invokeAsync(() -> createRegion());
+    AsyncInvocation server2RegionCreationInvocation = server2.invokeAsync(() -> createRegion());
+
+    // Wait for index creation to be in progress
+    server1.invoke(() -> waitForIndexCreationInProgress());
+    server2.invoke(() -> waitForIndexCreationInProgress());
+
+    // Execute command to destroy index
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --name=index0 --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FROM_REGION_1,
+        new Object[] {"index0", "/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Notify region creation to continue creating the region
+    server1.invoke(() -> notifyIndexDestroyComplete());
+    server2.invoke(() -> notifyIndexDestroyComplete());
+
+    server1RegionCreationInvocation.await(30, TimeUnit.SECONDS);
+    server2RegionCreationInvocation.await(30, TimeUnit.SECONDS);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  @Parameters({"true", "false"})
+  public void testDestroyIndexesWithOneIndex(boolean createRegion) throws Exception {
+    // Create index and region if necessary
+    server1.invoke(() -> createIndex(1));
+    server2.invoke(() -> createIndex(1));
+    if (createRegion) {
+      server1.invoke(() -> createRegion());
+      server2.invoke(() -> createRegion());
+    }
+
+    // Execute command to destroy indexes
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0,
+        new Object[] {"/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  public void testDestroyIndexesWithOneIndexAndRegionInOneMember() throws Exception {
+    // Create index in both members
+    server1.invoke(() -> createIndex(1));
+    server2.invoke(() -> createIndex(1));
+
+    // Create region in one member
+    server1.invoke(() -> createRegion());
+
+    // Execute command to destroy indexes
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0,
+        new Object[] {"/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  @Parameters({"true", "false"})
+  public void testDestroyIndexesWithTwoIndexes(boolean createRegion) throws Exception {
+    // Create index and region if necessary
+    server1.invoke(() -> createIndex(2));
+    server2.invoke(() -> createIndex(2));
+    if (createRegion) {
+      server1.invoke(() -> createRegion());
+      server2.invoke(() -> createRegion());
+    }
+
+    // Execute command to destroy indexes
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0,
+        new Object[] {"/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  public void testDestroyIndexesWithTwoIndexesAndRegionInOneMember() throws Exception {
+    // Create index in both members
+    server1.invoke(() -> createIndex(2));
+    server2.invoke(() -> createIndex(2));
+
+    // Create region in one member
+    server1.invoke(() -> createRegion());
+
+    // Execute command to destroy indexes
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0,
+        new Object[] {"/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  public void testDestroyNonExistentIndexes() throws Exception {
+    // Execute command to destroy indexes
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = LocalizedStrings.LuceneService_NO_INDEXES_WERE_FOUND_IN_REGION_0
+        .toLocalizedString("/region");
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  @Test
+  public void testDestroyIndexesWithRegionCreationInProgress() throws Exception {
+    server1.invoke(() -> initializeCountDownLatches());
+    server2.invoke(() -> initializeCountDownLatches());
+
+    server1.invoke(() -> createIndexesOnSpy(2));
+    server2.invoke(() -> createIndexesOnSpy(2));
+
+    // Asynchronously create region. This will cause the invokeBeforeAfterDataRegionCreated Answer
+    // to be invoked which will wait for the indexes to be destroyed before invoking the real
+    // afterDataRegionCreated method and completing region creation. The registerIndex method will
+    // realize the defined index has been destroyed and destroy the real one.
+    AsyncInvocation server1RegionCreationInvocation = server1.invokeAsync(() -> createRegion());
+    AsyncInvocation server2RegionCreationInvocation = server2.invokeAsync(() -> createRegion());
+
+    // Wait for index creation to be in progress
+    server1.invoke(() -> waitForIndexCreationInProgress());
+    server2.invoke(() -> waitForIndexCreationInProgress());
+
+    // Execute command to destroy index
+    CommandResultAssert commandResultAssert =
+        gfsh.executeAndAssertThat("destroy lucene index --region=region");
+
+    // Assert command was successful and contains the correct rows and output
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0,
+        new Object[] {"/region"});
+    validateCommandResult(commandResultAssert, expectedStatus);
+
+    // Notify region creation to continue creating the region
+    server1.invoke(() -> notifyIndexDestroyComplete());
+    server2.invoke(() -> notifyIndexDestroyComplete());
+
+    server1RegionCreationInvocation.await(30, TimeUnit.SECONDS);
+    server2RegionCreationInvocation.await(30, TimeUnit.SECONDS);
+
+    // Verify defined and created indexes are empty in both members
+    server1.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+    server2.invoke(() -> verifyDefinedAndCreatedIndexSizes(0, 0));
+  }
+
+  private void validateCommandResult(CommandResultAssert commandResultAssert,
+      String expectedStatus) {
+    commandResultAssert.statusIsSuccess();
+    commandResultAssert.tableHasRowCount("Member", 2);
+    commandResultAssert.tableHasColumnOnlyWithValues("Status", expectedStatus);
+  }
+
+  private void verifyDefinedAndCreatedIndexSizes(int definedIndexesSize, int createdIndexesSize) {
+    LuceneServiceImpl luceneService =
+        (LuceneServiceImpl) LuceneServiceProvider.get(ClusterStartupRule.getCache());
+    assertThat(luceneService.getIndexes(REGION_NAME)).isEmpty();
+    assertThat(luceneService.getDefinedIndexes(REGION_NAME)).isEmpty();
+  }
+
+  private void createIndex(int numIndexes) {
+    LuceneService luceneService = LuceneServiceProvider.get(ClusterStartupRule.getCache());
+    for (int i = 0; i < numIndexes; i++) {
+      luceneService.createIndexFactory().setFields("text" + i).create(INDEX_NAME + i, REGION_NAME);
+    }
+  }
+
+  private void createRegion() {
+    ClusterStartupRule.getCache().createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+  }
+
+  private void initializeCountDownLatches() {
+    indexCreationInProgress = new CountDownLatch(1);
+    indexDestroyComplete = new CountDownLatch(1);
+  }
+
+  private void createIndexesOnSpy(int numIndexes) {
+    LuceneServiceImpl luceneServiceSpy =
+        (LuceneServiceImpl) spy(LuceneServiceProvider.get(ClusterStartupRule.getCache()));
+    for (int i = 0; i < numIndexes; i++) {
+      luceneServiceSpy.createIndexFactory().setFields("text" + i).create(INDEX_NAME + i,
+          REGION_NAME);
+    }
+
+    Answer invokeBeforeAfterDataRegionCreated = invocation -> {
+      // Confirm index creation is in progress
+      indexCreationInProgress.countDown();
+
+      // Wait for destroy index invocation to complete
+      indexDestroyComplete.await();
+
+      return invocation.callRealMethod();
+    };
+
+    doAnswer(invokeBeforeAfterDataRegionCreated).when(luceneServiceSpy)
+        .afterDataRegionCreated(any());
+  }
+
+  private void waitForIndexCreationInProgress() throws Exception {
+    indexCreationInProgress.await();
+  }
+
+  private void notifyIndexDestroyComplete() throws Exception {
+    indexDestroyComplete.countDown();
+  }
+}
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
index bded765..1242f1f 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
@@ -643,12 +643,20 @@ public class LuceneIndexCommandsDUnitTest implements Serializable {
       createIndexWithoutRegion();
     }
 
+    // Verify destroy all indexes is successful
     String expectedOutput = CliStrings.format(
         LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0,
         new Object[] {"/region"});
 
     gfsh.executeAndAssertThat("destroy lucene index --region=region").statusIsSuccess()
         .containsOutput(expectedOutput);
+
+    // Verify destroy all indexes again reports no indexes exist
+    expectedOutput = LocalizedStrings.LuceneService_NO_INDEXES_WERE_FOUND_IN_REGION_0
+        .toLocalizedString(new Object[] {"/region"});
+
+    gfsh.executeAndAssertThat("destroy lucene index --region=region").statusIsSuccess()
+        .containsOutput(expectedOutput);
   }
 
   @Test
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
index 9f6bdc0..46a7e5e 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.cache.lucene.internal.cli;
 
+import static org.apache.commons.lang.SystemUtils.LINE_SEPARATOR;
+import static org.apache.geode.management.internal.cli.result.ResultData.TYPE_TABULAR;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.any;
@@ -59,6 +61,7 @@ import org.apache.geode.cache.lucene.internal.cli.functions.LuceneListIndexFunct
 import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import org.apache.geode.cache.lucene.internal.repository.serializer.PrimitiveSerializer;
 import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.internal.security.SecurityService;
@@ -427,34 +430,15 @@ public class LuceneIndexCommandsJUnitTest {
   }
 
   @Test
-  @Parameters({"true", "false"})
-  public void testDestroySingleIndexNoRegionMembers(boolean expectedToSucceed) throws Exception {
+  public void testDestroySingleIndexNoRegionMembers() throws Exception {
     LuceneIndexCommands commands = createTestLuceneIndexCommandsForDestroyIndex();
-    String indexName = "index";
-    String regionPath = "regionPath";
-
-    final ResultCollector mockResultCollector = mock(ResultCollector.class);
     final List<CliFunctionResult> cliFunctionResults = new ArrayList<>();
-    String expectedStatus;
-    if (expectedToSucceed) {
-      expectedStatus = CliStrings.format(
-          LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FROM_REGION_1,
-          new Object[] {indexName, regionPath});
-      cliFunctionResults.add(new CliFunctionResult("member0"));
-    } else {
-      Exception e = new IllegalStateException("failed");
-      expectedStatus = e.getMessage();
-      cliFunctionResults.add(new CliFunctionResult("member0", e, e.getMessage()));
-    }
-
-    doReturn(mockResultCollector).when(commands).executeFunction(
-        isA(LuceneDestroyIndexFunction.class), any(LuceneDestroyIndexInfo.class), any(Set.class));
-    doReturn(cliFunctionResults).when(mockResultCollector).getResult();
-
-    doReturn(Collections.emptySet()).when(commands).getAllNormalMembers();
-    doReturn(Collections.emptySet()).when(commands).findMembersForRegion(any());
-
-    CommandResult result = (CommandResult) commands.destroyIndex(indexName, regionPath);
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__COULD_NOT_FIND__MEMBERS_GREATER_THAN_VERSION_0,
+        new Object[] {Version.GEODE_160}) + LINE_SEPARATOR;
+    cliFunctionResults.add(new CliFunctionResult("member0"));
+    doReturn(Collections.emptySet()).when(commands).getNormalMembersWithSameOrNewerVersion(any());
+    CommandResult result = (CommandResult) commands.destroyIndex("index", "regionPath");
     verifyDestroyIndexCommandResult(result, cliFunctionResults, expectedStatus);
   }
 
@@ -488,42 +472,22 @@ public class LuceneIndexCommandsJUnitTest {
         isA(LuceneDestroyIndexFunction.class), any(LuceneDestroyIndexInfo.class), any(Set.class));
     doReturn(cliFunctionResults).when(mockResultCollector).getResult();
 
-    doReturn(members).when(commands).getAllNormalMembers();
-    doReturn(members).when(commands).findMembersForRegion(any());
+    doReturn(members).when(commands).getNormalMembersWithSameOrNewerVersion(any());
 
     CommandResult result = (CommandResult) commands.destroyIndex(indexName, regionPath);
     verifyDestroyIndexCommandResult(result, cliFunctionResults, expectedStatus);
   }
 
   @Test
-  @Parameters({"true", "false"})
-  public void testDestroyAllIndexesNoRegionMembers(boolean expectedToSucceed) throws Exception {
+  public void testDestroyAllIndexesNoRegionMembers() throws Exception {
     LuceneIndexCommands commands = createTestLuceneIndexCommandsForDestroyIndex();
-    String indexName = null;
-    String regionPath = "regionPath";
-
-    final ResultCollector mockResultCollector = mock(ResultCollector.class);
+    doReturn(Collections.emptySet()).when(commands).getNormalMembersWithSameOrNewerVersion(any());
     final List<CliFunctionResult> cliFunctionResults = new ArrayList<>();
-    String expectedStatus;
-    if (expectedToSucceed) {
-      expectedStatus = CliStrings.format(
-          LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FROM_REGION_0,
-          new Object[] {regionPath});
-      cliFunctionResults.add(new CliFunctionResult("member0"));
-    } else {
-      Exception e = new IllegalStateException("failed");
-      expectedStatus = e.getMessage();
-      cliFunctionResults.add(new CliFunctionResult("member0", e, e.getMessage()));
-    }
-
-    doReturn(mockResultCollector).when(commands).executeFunction(
-        isA(LuceneDestroyIndexFunction.class), any(LuceneDestroyIndexInfo.class), any(Set.class));
-    doReturn(cliFunctionResults).when(mockResultCollector).getResult();
-
-    doReturn(Collections.emptySet()).when(commands).getAllNormalMembers();
-    doReturn(Collections.emptySet()).when(commands).findMembersForRegion(any());
-
-    CommandResult result = (CommandResult) commands.destroyIndex(indexName, regionPath);
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__COULD_NOT_FIND__MEMBERS_GREATER_THAN_VERSION_0,
+        new Object[] {Version.GEODE_160}) + LINE_SEPARATOR;
+    cliFunctionResults.add(new CliFunctionResult("member0"));
+    CommandResult result = (CommandResult) commands.destroyIndex(null, "regionPath");
     verifyDestroyIndexCommandResult(result, cliFunctionResults, expectedStatus);
   }
 
@@ -557,8 +521,7 @@ public class LuceneIndexCommandsJUnitTest {
         isA(LuceneDestroyIndexFunction.class), any(LuceneDestroyIndexInfo.class), any(Set.class));
     doReturn(cliFunctionResults).when(mockResultCollector).getResult();
 
-    doReturn(Collections.emptySet()).when(commands).getAllNormalMembers();
-    doReturn(Collections.emptySet()).when(commands).findMembersForRegion(any());
+    doReturn(members).when(commands).getNormalMembersWithSameOrNewerVersion(any());
 
     CommandResult result = (CommandResult) commands.destroyIndex(indexName, regionPath);
     verifyDestroyIndexCommandResult(result, cliFunctionResults, expectedStatus);
@@ -580,17 +543,22 @@ public class LuceneIndexCommandsJUnitTest {
   private void verifyDestroyIndexCommandResult(CommandResult result,
       List<CliFunctionResult> cliFunctionResults, String expectedStatus) {
     assertEquals(Status.OK, result.getStatus());
-    TabularResultData data = (TabularResultData) result.getResultData();
-    List<String> members = data.retrieveAllValues("Member");
-    assertEquals(cliFunctionResults.size(), members.size());
-    // Verify each member
-    for (int i = 0; i < members.size(); i++) {
-      assertEquals("member" + i, members.get(i));
-    }
-    // Verify each status
-    List<String> status = data.retrieveAllValues("Status");
-    for (String statu : status) {
-      assertEquals(expectedStatus, statu);
+    if (result.getType().equals(TYPE_TABULAR)) {
+      TabularResultData data = (TabularResultData) result.getResultData();
+      List<String> members = data.retrieveAllValues("Member");
+      assertEquals(cliFunctionResults.size(), members.size());
+      // Verify each member
+      for (int i = 0; i < members.size(); i++) {
+        assertEquals("member" + i, members.get(i));
+      }
+      // Verify each status
+      List<String> status = data.retrieveAllValues("Status");
+      for (String statu : status) {
+        assertEquals(expectedStatus, statu);
+      }
+    } else {
+      // Info result. Verify next lines are equal.
+      assertEquals(result.nextLine(), expectedStatus);
     }
   }
 
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
index 6b083f4..3928ee5 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
@@ -134,8 +134,8 @@ public class LuceneDestroyIndexFunctionJUnitTest {
     function = spy(function);
     function.execute(this.context);
     verify(this.service).destroyIndexes(eq(regionPath));
+    verify(this.service).destroyDefinedIndexes(eq(regionPath));
     verify(function).getXmlEntity(eq(null), eq(regionPath));
-    verify(this.service, never()).destroyDefinedIndexes(eq(regionPath));
     verify(this.service, never()).destroyIndex(any(), eq(regionPath));
     verifyFunctionResult(true);
   }
@@ -148,6 +148,7 @@ public class LuceneDestroyIndexFunctionJUnitTest {
     when(this.context.getArguments()).thenReturn(indexInfo);
     LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
     doThrow(new IllegalStateException()).when(this.service).destroyIndexes(eq(regionPath));
+    doThrow(new IllegalStateException()).when(this.service).destroyDefinedIndexes(eq(regionPath));
     function.execute(this.context);
     verifyFunctionResult(false);
   }

-- 
To stop receiving notification emails like this one, please contact
boglesby@apache.org.