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

[29/34] geode git commit: GEODE-2404: Added support for destroying lucene indexes

GEODE-2404: Added support for destroying lucene indexes


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

Branch: refs/heads/GEODE-4160-mockito
Commit: 11521a824f31ff03db13d0e59cb0fbf29e592151
Parents: c4a5ab2
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Mon Feb 13 17:01:13 2017 -0800
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Mon Feb 27 10:06:48 2017 -0800

----------------------------------------------------------------------
 .../internal/AsyncEventQueueImpl.java           |   9 +
 .../geode/internal/DataSerializableFixedID.java |   2 +-
 .../geode/internal/i18n/LocalizedStrings.java   |   4 +
 .../geode/cache/lucene/LuceneService.java       |  14 +-
 .../internal/DestroyLuceneIndexMessage.java     | 109 ++++++++
 .../lucene/internal/InternalLuceneIndex.java    |   5 +
 .../LuceneIndexForPartitionedRegion.java        | 109 +++++++-
 .../cache/lucene/internal/LuceneIndexImpl.java  |  16 ++
 .../cache/lucene/internal/LuceneRawIndex.java   |   2 +
 .../lucene/internal/LuceneServiceImpl.java      |  57 ++++-
 .../lucene/internal/cli/LuceneCliStrings.java   |  28 ++-
 .../cli/LuceneFunctionSerializable.java         |  36 +++
 .../internal/cli/LuceneIndexCommands.java       | 113 +++++++--
 .../lucene/internal/cli/LuceneIndexDetails.java |  16 +-
 .../lucene/internal/cli/LuceneIndexInfo.java    |  15 +-
 .../lucene/internal/cli/LuceneQueryInfo.java    |  15 +-
 .../functions/LuceneDestroyIndexFunction.java   |  57 +++++
 .../lucene/LuceneIndexDestroyDUnitTest.java     | 247 +++++++++++++++++++
 .../cli/LuceneIndexCommandsDUnitTest.java       |  50 +++-
 .../cli/LuceneIndexCommandsJUnitTest.java       |  53 ++++
 .../LuceneDestroyIndexFunctionJUnitTest.java    |  93 +++++++
 .../LuceneClusterConfigurationDUnitTest.java    |  99 +++++++-
 22 files changed, 1068 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
index 3b99f1c..a44b9e4 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
@@ -25,6 +25,7 @@ import org.apache.geode.cache.wan.GatewayEventFilter;
 import org.apache.geode.cache.wan.GatewayEventSubstitutionFilter;
 import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.cache.wan.GatewaySender.OrderPolicy;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.RegionQueue;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor;
@@ -191,8 +192,16 @@ public class AsyncEventQueueImpl implements AsyncEventQueue {
     return ((AbstractGatewaySender) sender).getIsMetaQueue();
   }
 
+  public void stop() {
+    if (this.sender.isRunning()) {
+      this.sender.stop();
+    }
+  }
+
   public void destroy() {
+    GemFireCacheImpl gfci = (GemFireCacheImpl) ((AbstractGatewaySender) this.sender).getCache();
     this.sender.destroy();
+    gfci.removeAsyncEventQueue(this);
   }
 
   public boolean isBucketSorted() {

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java b/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java
index 74e40e9..4e45646 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java
@@ -809,8 +809,8 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final short LUCENE_ENTRY_SCORE = 2174;
   public static final short LUCENE_TOP_ENTRIES = 2175;
   public static final short LUCENE_TOP_ENTRIES_COLLECTOR = 2176;
-
   public static final short WAIT_UNTIL_FLUSHED_FUNCTION_CONTEXT = 2177;
+  public static final short DESTROY_LUCENE_INDEX_MESSAGE = 2178;
 
   // NOTE, codes > 65535 will take 4 bytes to serialize
 

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index d855e29..fa63437 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -7674,6 +7674,10 @@ public class LocalizedStrings {
       new StringId(6650,
           "Caught the following exception attempting waitUntilFlushed and will return:");
 
+  public static final StringId LuceneService_INDEX_0_NOT_FOUND_IN_REGION_1 =
+      new StringId(6651, "Lucene index {0} was not found in region {1}.");
+  public static final StringId LuceneService_DESTROYED_INDEX_0_FROM_REGION_1 =
+      new StringId(6652, "Destroyed Lucene index {0} from region {1}.");
   /** Testing strings, messageId 90000-99999 **/
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
index d273760..5cfae59 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneService.java
@@ -101,10 +101,18 @@ public interface LuceneService {
 
   /**
    * Destroy the lucene index
-   * 
-   * @param index index object
+   *
+   * @param indexName the name of the index to destroy
+   * @param regionPath the path of the region whose index to destroy
+   */
+  public void destroyIndex(String indexName, String regionPath);
+
+  /**
+   * Destroy all the lucene indexes for the region
+   *
+   * @param regionPath The path of the region on which to destroy the indexes
    */
-  public void destroyIndex(LuceneIndex index);
+  public void destroyIndexes(String regionPath);
 
   /**
    * Get the lucene index object specified by region name and index name

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/DestroyLuceneIndexMessage.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8bdef9b
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/DestroyLuceneIndexMessage.java
@@ -0,0 +1,109 @@
+/*
+ * 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;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.lucene.LuceneServiceProvider;
+import org.apache.geode.distributed.internal.*;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+
+public class DestroyLuceneIndexMessage extends PooledDistributionMessage
+    implements MessageWithReply {
+
+  private int processorId;
+
+  private String regionPath;
+
+  private String indexName;
+
+  private static final Logger logger = LogService.getLogger();
+
+  /* For serialization */
+  public DestroyLuceneIndexMessage() {}
+
+  protected DestroyLuceneIndexMessage(Collection recipients, int processorId, String regionPath,
+      String indexName) {
+    super();
+    setRecipients(recipients);
+    this.processorId = processorId;
+    this.regionPath = regionPath;
+    this.indexName = indexName;
+  }
+
+  @Override
+  protected void process(DistributionManager dm) {
+    ReplyException replyException = null;
+    try {
+      if (logger.isDebugEnabled()) {
+        logger.debug("DestroyLuceneIndexMessage: Destroying regionPath=" + this.regionPath
+            + "; indexName=" + this.indexName);
+      }
+      try {
+        Cache cache = GemFireCacheImpl.getInstance();
+        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);
+        }
+      } catch (Throwable e) {
+        replyException = new ReplyException(e);
+        if (logger.isDebugEnabled()) {
+          logger.debug(
+              "DestroyLuceneIndexMessage: Caught the following exception attempting to destroy indexName="
+                  + this.indexName + "; regionPath=" + this.regionPath + ":",
+              e);
+        }
+      }
+    } finally {
+      ReplyMessage replyMsg = new ReplyMessage();
+      replyMsg.setRecipient(getSender());
+      replyMsg.setProcessorId(this.processorId);
+      if (replyException != null) {
+        replyMsg.setException(replyException);
+      }
+      dm.putOutgoing(replyMsg);
+    }
+  }
+
+  @Override
+  public int getDSFID() {
+    return DESTROY_LUCENE_INDEX_MESSAGE;
+  }
+
+  @Override
+  public void toData(DataOutput out) throws IOException {
+    super.toData(out);
+    out.writeInt(this.processorId);
+    DataSerializer.writeString(this.regionPath, out);
+    DataSerializer.writeString(this.indexName, out);
+  }
+
+  @Override
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    super.fromData(in);
+    this.processorId = in.readInt();
+    this.regionPath = DataSerializer.readString(in);
+    this.indexName = DataSerializer.readString(in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/InternalLuceneIndex.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/InternalLuceneIndex.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/InternalLuceneIndex.java
index 2800f49..c28fcdc 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/InternalLuceneIndex.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/InternalLuceneIndex.java
@@ -27,4 +27,9 @@ public interface InternalLuceneIndex extends LuceneIndex {
    */
   public void dumpFiles(String directory);
 
+  /**
+   * Destroy the index
+   */
+  public void destroy(boolean initiator);
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
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 9a39b39..f45d94d 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
@@ -15,6 +15,7 @@
 
 package org.apache.geode.cache.lucene.internal;
 
+import org.apache.geode.CancelException;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.FixedPartitionResolver;
@@ -24,6 +25,7 @@ import org.apache.geode.cache.PartitionResolver;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.lucene.internal.directory.DumpDirectoryFiles;
@@ -36,15 +38,23 @@ import org.apache.geode.cache.lucene.internal.repository.RepositoryManager;
 import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import org.apache.geode.cache.partition.PartitionListener;
 import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ReplyException;
+import org.apache.geode.distributed.internal.ReplyProcessor21;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.PartitionedRegion;
 
+import java.util.Set;
+
 /* wrapper of IndexWriter */
 public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   protected Region<String, File> fileRegion;
   protected Region<ChunkKey, byte[]> chunkRegion;
   protected final FileSystemStats fileSystemStats;
 
+  public static final String FILES_REGION_SUFFIX = ".files";
+  public static final String CHUNKS_REGION_SUFFIX = ".chunks";
+
   public LuceneIndexForPartitionedRegion(String indexName, String regionPath, Cache cache) {
     super(indexName, regionPath, cache);
 
@@ -123,7 +133,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   public String createFileRegionName() {
-    return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".files";
+    return LuceneServiceImpl.getUniqueIndexRegionName(indexName, regionPath, FILES_REGION_SUFFIX);
   }
 
   boolean chunkRegionExists(String chunkRegionName) {
@@ -139,7 +149,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   public String createChunkRegionName() {
-    return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
+    return LuceneServiceImpl.getUniqueIndexRegionName(indexName, regionPath, CHUNKS_REGION_SUFFIX);
   }
 
   private PartitionAttributesFactory configureLuceneRegionAttributesFactory(
@@ -192,4 +202,99 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
         .withArgs(new String[] {directory, indexName}).execute(DumpDirectoryFiles.ID);
     results.getResult();
   }
+
+  @Override
+  public void destroy(boolean initiator) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("Destroying index regionPath=" + regionPath + "; indexName=" + indexName
+          + "; initiator=" + initiator);
+    }
+
+    // Invoke super destroy to remove the extension
+    super.destroy(initiator);
+
+    // Destroy the AsyncEventQueue
+    PartitionedRegion pr = (PartitionedRegion) getDataRegion();
+    destroyAsyncEventQueue(pr);
+
+    // Destroy the chunk region (colocated with the file region)
+    // localDestroyRegion can't be used because locally destroying regions is not supported on
+    // colocated regions
+    if (!chunkRegion.isDestroyed()) {
+      chunkRegion.destroyRegion();
+      if (logger.isDebugEnabled()) {
+        logger.debug("Destroyed chunkRegion=" + chunkRegion.getName());
+      }
+    }
+
+    // Destroy the file region (colocated with the application region)
+    // localDestroyRegion can't be used because locally destroying regions is not supported on
+    // colocated regions
+    if (!fileRegion.isDestroyed()) {
+      fileRegion.destroyRegion();
+      if (logger.isDebugEnabled()) {
+        logger.debug("Destroyed fileRegion=" + fileRegion.getName());
+      }
+    }
+
+    // Destroy index on remote members if necessary
+    if (initiator) {
+      destroyOnRemoteMembers(pr);
+    }
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("Destroyed index regionPath=" + regionPath + "; indexName=" + indexName
+          + "; initiator=" + initiator);
+    }
+  }
+
+  private void destroyAsyncEventQueue(PartitionedRegion pr) {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
+
+    // Get the AsyncEventQueue
+    AsyncEventQueueImpl aeq = (AsyncEventQueueImpl) cache.getAsyncEventQueue(aeqId);
+
+    // Stop the AsyncEventQueue (this stops the AsyncEventQueue's underlying GatewaySender)
+    aeq.stop();
+
+    // Remove the id from the dataRegion's AsyncEventQueue ids
+    // Note: The region may already have been destroyed by a remote member
+    if (!pr.isDestroyed()) {
+      pr.getAttributesMutator().removeAsyncEventQueueId(aeqId);
+    }
+
+    // Destroy the aeq (this also removes it from the GemFireCacheImpl)
+    aeq.destroy();
+    if (logger.isDebugEnabled()) {
+      logger.debug("Destroyed aeqId=" + aeqId);
+    }
+  }
+
+  private void destroyOnRemoteMembers(PartitionedRegion pr) {
+    DM dm = pr.getDistributionManager();
+    Set<InternalDistributedMember> recipients = pr.getRegionAdvisor().adviseDataStore();
+    if (!recipients.isEmpty()) {
+      if (logger.isDebugEnabled()) {
+        logger.debug("LuceneIndexForPartitionedRegion: About to send destroy message recipients="
+            + recipients);
+      }
+      ReplyProcessor21 processor = new ReplyProcessor21(dm, recipients);
+      DestroyLuceneIndexMessage message = new DestroyLuceneIndexMessage(recipients,
+          processor.getProcessorId(), regionPath, indexName);
+      dm.putOutgoing(message);
+      if (logger.isDebugEnabled()) {
+        logger.debug("LuceneIndexForPartitionedRegion: Sent message recipients=" + recipients);
+      }
+      try {
+        processor.waitForReplies();
+      } catch (ReplyException e) {
+        if (!(e.getCause() instanceof CancelException)) {
+          throw e;
+        }
+      } catch (InterruptedException e) {
+        dm.getCancelCriterion().checkCancelInProgress(e);
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
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 42ccc84..cf519be 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
@@ -20,6 +20,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.geode.internal.cache.extension.Extension;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
@@ -203,6 +204,21 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     dataRegion.getExtensionPoint().addExtension(creation);
   }
 
+  public void destroy(boolean initiator) {
+    // Find and delete the appropriate extension
+    Extension extensionToDelete = null;
+    for (Extension extension : getDataRegion().getExtensionPoint().getExtensions()) {
+      LuceneIndexCreation index = (LuceneIndexCreation) extension;
+      if (index.getName().equals(indexName)) {
+        extensionToDelete = extension;
+        break;
+      }
+    }
+    if (extensionToDelete != null) {
+      getDataRegion().getExtensionPoint().removeExtension(extensionToDelete);
+    }
+  }
+
   protected <K, V> Region<K, V> createRegion(final String regionName,
       final RegionAttributes<K, V> attributes) {
     // Create InternalRegionArguments to set isUsedForMetaRegion true to suppress xml generation

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndex.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndex.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndex.java
index a4e48ad..f4518aa 100755
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndex.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndex.java
@@ -36,4 +36,6 @@ public class LuceneRawIndex extends LuceneIndexImpl {
     return;
   }
 
+  @Override
+  public void destroy(boolean initiator) {}
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
----------------------------------------------------------------------
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 cf7b2c9..a608dd9 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
@@ -114,6 +114,11 @@ public class LuceneServiceImpl implements InternalLuceneService {
     return name;
   }
 
+  public static String getUniqueIndexRegionName(String indexName, String regionPath,
+      String regionSuffix) {
+    return getUniqueIndexName(indexName, regionPath) + regionSuffix;
+  }
+
   @Override
   public void createIndex(String indexName, String regionPath, String... fields) {
     if (fields == null || fields.length == 0) {
@@ -256,10 +261,53 @@ public class LuceneServiceImpl implements InternalLuceneService {
   }
 
   @Override
-  public void destroyIndex(LuceneIndex index) {
-    LuceneIndexImpl indexImpl = (LuceneIndexImpl) index;
+  public void destroyIndex(String indexName, String regionPath) {
+    destroyIndex(indexName, regionPath, true);
+  }
+
+  protected void destroyIndex(String indexName, String regionPath, boolean initiator) {
+    if (!regionPath.startsWith("/")) {
+      regionPath = "/" + regionPath;
+    }
+    LuceneIndexImpl indexImpl = (LuceneIndexImpl) getIndex(indexName, regionPath);
+    if (indexImpl == null) {
+      throw new IllegalArgumentException(
+          LocalizedStrings.LuceneService_INDEX_0_NOT_FOUND_IN_REGION_1.toLocalizedString(indexName,
+              regionPath));
+    } else {
+      indexImpl.destroy(initiator);
+      removeFromIndexMap(indexImpl);
+      logger.info(LocalizedStrings.LuceneService_DESTROYED_INDEX_0_FROM_REGION_1
+          .toLocalizedString(indexName, regionPath));
+    }
+  }
+
+  @Override
+  public void destroyIndexes(String regionPath) {
+    destroyIndexes(regionPath, true);
+  }
+
+  protected void destroyIndexes(String regionPath, boolean initiator) {
+    if (!regionPath.startsWith("/")) {
+      regionPath = "/" + regionPath;
+    }
+    List<LuceneIndexImpl> indexesToDestroy = new ArrayList<>();
+    for (LuceneIndex index : getAllIndexes()) {
+      if (index.getRegionPath().equals(regionPath)) {
+        LuceneIndexImpl indexImpl = (LuceneIndexImpl) index;
+        indexImpl.destroy(initiator);
+        indexesToDestroy.add(indexImpl);
+      }
+    }
+    for (LuceneIndex index : indexesToDestroy) {
+      removeFromIndexMap(index);
+      logger.info(LocalizedStrings.LuceneService_DESTROYED_INDEX_0_FROM_REGION_1
+          .toLocalizedString(index.getName(), regionPath));
+    }
+  }
+
+  private void removeFromIndexMap(LuceneIndex index) {
     indexMap.remove(getUniqueIndexName(index.getName(), index.getRegionPath()));
-    // indexImpl.close();
   }
 
   @Override
@@ -320,6 +368,9 @@ public class LuceneServiceImpl implements InternalLuceneService {
 
     DSFIDFactory.registerDSFID(DataSerializableFixedID.WAIT_UNTIL_FLUSHED_FUNCTION_CONTEXT,
         WaitUntilFlushedFunctionContext.class);
+
+    DSFIDFactory.registerDSFID(DataSerializableFixedID.DESTROY_LUCENE_INDEX_MESSAGE,
+        DestroyLuceneIndexMessage.class);
   }
 
   public Collection<LuceneIndexCreationProfile> getAllDefinedIndexes() {

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneCliStrings.java
----------------------------------------------------------------------
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 36f5aeb..fbb70d2 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
@@ -37,7 +37,7 @@ public class LuceneCliStrings {
   public static final String LUCENE_CREATE_INDEX__NAME__HELP =
       "Name of the lucene index to create.";
   public static final String LUCENE_CREATE_INDEX__REGION_HELP =
-      "Name/Path of the region where the lucene index is created on.";
+      "Name/Path of the region on which to create the lucene index.";
   public static final String LUCENE_CREATE_INDEX__FIELD = "field";
   public static final String LUCENE_CREATE_INDEX__FIELD_HELP =
       "fields on the region values which are stored in the lucene index.";
@@ -61,14 +61,13 @@ public class LuceneCliStrings {
   // Describe lucene index commands
   public static final String LUCENE_DESCRIBE_INDEX = "describe lucene index";
   public static final String LUCENE_DESCRIBE_INDEX__HELP =
-      "Display the describe of lucene indexes created for all members.";
+      "Display the description of lucene indexes created for all members.";
   public static final String LUCENE_DESCRIBE_INDEX__ERROR_MESSAGE =
       "An error occurred while collecting lucene index information across the Geode cluster: %1$s";
   public static final String LUCENE_DESCRIBE_INDEX__NAME__HELP =
       "Name of the lucene index to describe.";
   public static final String LUCENE_DESCRIBE_INDEX__REGION_HELP =
-      "Name/Path of the region where the lucene index to be described exists.";
-
+      "Name/Path of the region defining the lucene index to be described.";
 
   // Search lucene index commands
   public static final String LUCENE_SEARCH_INDEX = "search lucene";
@@ -78,7 +77,7 @@ public class LuceneCliStrings {
   public static final String LUCENE_SEARCH_INDEX__NAME__HELP =
       "Name of the lucene index to search.";
   public static final String LUCENE_SEARCH_INDEX__REGION_HELP =
-      "Name/Path of the region where the lucene index exists.";
+      "Name/Path of the region defining the lucene index to be searched.";
   public static final String LUCENE_SEARCH_INDEX__QUERY_STRING = "queryStrings";
   public static final String LUCENE_SEARCH_INDEX__LIMIT = "limit";
   public static final String LUCENE_SEARCH_INDEX__LIMIT__HELP = "Number of search results needed";
@@ -95,4 +94,23 @@ public class LuceneCliStrings {
   public static final String LUCENE_SEARCH_INDEX__KEYSONLY__HELP =
       "Return only keys of search results.";
 
+  // Destroy lucene index command
+  public static final String LUCENE_DESTROY_INDEX = "destroy lucene index";
+  public static final String LUCENE_DESTROY_INDEX__HELP = "Destroy the lucene index.";
+  public static final String LUCENE_DESTROY_INDEX__EXCEPTION_MESSAGE =
+      "An unexpected exception occurred while destroying lucene index:";
+  public static final String LUCENE_DESTROY_INDEX__NAME__HELP =
+      "Name of the lucene index to destroy.";
+  public static final String LUCENE_DESTROY_INDEX__REGION_HELP =
+      "Name of the region defining the lucene index to be destroyed.";
+  public static final String LUCENE_DESTROY_INDEX__MSG__REGION_CANNOT_BE_EMPTY =
+      "Region cannot be empty.";
+  public static final String LUCENE_DESTROY_INDEX__MSG__INDEX_CANNOT_BE_EMPTY =
+      "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__SUCCESSFULLY_DESTROYED_INDEXES_FOR_REGION_0 =
+      "Successfully destroyed all lucene indexes for region {0}";
+  public static final String LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FOR_REGION_1 =
+      "Successfully destroyed lucene index {0} for region {1}";
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneFunctionSerializable.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneFunctionSerializable.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneFunctionSerializable.java
new file mode 100644
index 0000000..1390e22
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneFunctionSerializable.java
@@ -0,0 +1,36 @@
+/*
+ * 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 java.io.Serializable;
+
+public class LuceneFunctionSerializable implements Serializable {
+
+  protected final String indexName;
+  protected final String regionPath;
+
+  public LuceneFunctionSerializable(final String indexName, final String regionPath) {
+    this.indexName = indexName;
+    this.regionPath = regionPath;
+  }
+
+  public String getIndexName() {
+    return indexName;
+  }
+
+  public String getRegionPath() {
+    return regionPath;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
----------------------------------------------------------------------
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 b84246c..e2d85a6 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
@@ -16,16 +16,12 @@ package org.apache.geode.cache.lucene.internal.cli;
 
 import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.execute.Execution;
-import org.apache.geode.cache.execute.FunctionAdapter;
-import org.apache.geode.cache.execute.FunctionInvocationTargetException;
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.cache.lucene.internal.cli.functions.LuceneCreateIndexFunction;
-import org.apache.geode.cache.lucene.internal.cli.functions.LuceneDescribeIndexFunction;
-import org.apache.geode.cache.lucene.internal.cli.functions.LuceneListIndexFunction;
-import org.apache.geode.cache.lucene.internal.cli.functions.LuceneSearchIndexFunction;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.*;
+import org.apache.geode.cache.lucene.internal.cli.functions.*;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
+import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.internal.security.IntegratedSecurityService;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.management.cli.CliMetaData;
@@ -70,6 +66,8 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
       new LuceneDescribeIndexFunction();
   private static final LuceneSearchIndexFunction searchIndexFunction =
       new LuceneSearchIndexFunction();
+  private static final LuceneDestroyIndexFunction destroyIndexFunction =
+      new LuceneDestroyIndexFunction();
   private List<LuceneSearchResults> searchResults = null;
 
   private SecurityService securityService = IntegratedSecurityService.getSecurityService();
@@ -316,6 +314,74 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
     }
   }
 
+  @CliCommand(value = LuceneCliStrings.LUCENE_DESTROY_INDEX,
+      help = LuceneCliStrings.LUCENE_DESTROY_INDEX__HELP)
+  @CliMetaData(shellOnly = false,
+      relatedTopic = {CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA})
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
+  public Result destroyIndex(
+      @CliOption(key = LuceneCliStrings.LUCENE__INDEX_NAME, mandatory = false,
+          help = LuceneCliStrings.LUCENE_DESTROY_INDEX__NAME__HELP) final String indexName,
+
+      @CliOption(key = LuceneCliStrings.LUCENE__REGION_PATH, mandatory = true,
+          optionContext = ConverterHint.REGIONPATH,
+          help = LuceneCliStrings.LUCENE_DESTROY_INDEX__REGION_HELP) final String regionPath) {
+    if (StringUtils.isBlank(regionPath) || regionPath.equals(Region.SEPARATOR)) {
+      return ResultBuilder.createInfoResult(
+          CliStrings.format(LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__REGION_CANNOT_BE_EMPTY));
+    }
+
+    if (StringUtils.isEmpty(indexName)) {
+      return ResultBuilder.createInfoResult(
+          CliStrings.format(LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__INDEX_CANNOT_BE_EMPTY));
+    }
+
+    this.securityService.authorizeRegionManage(regionPath);
+
+    Result result = null;
+    try {
+      LuceneIndexInfo indexInfo = new LuceneIndexInfo(indexName, regionPath);
+      ResultCollector<?, ?> rc = executeFunction(destroyIndexFunction, indexInfo, false);
+      List<CliFunctionResult> functionResults = (List<CliFunctionResult>) rc.getResult();
+      CliFunctionResult cliFunctionResult = functionResults.get(0);
+
+      final TabularResultData tabularResult = ResultBuilder.createTabularResultData();
+      tabularResult.accumulate("Member", cliFunctionResult.getMemberIdOrName());
+      if (cliFunctionResult.isSuccessful()) {
+        tabularResult.accumulate("Status",
+            indexName == null
+                ? CliStrings.format(
+                    LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FOR_REGION_0,
+                    new Object[] {regionPath})
+                : CliStrings.format(
+                    LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FOR_REGION_1,
+                    new Object[] {indexName, regionPath}));
+      } else {
+        tabularResult.accumulate("Status", "Failed: " + cliFunctionResult.getMessage());
+      }
+      result = ResultBuilder.buildResult(tabularResult);
+      if (cliFunctionResult.isSuccessful()) {
+        persistClusterConfiguration(result, () -> {
+          // Update the xml entity (region entity) to remove the async event id(s) and index(es)
+          getSharedConfiguration().addXmlEntity((XmlEntity) cliFunctionResult.getXmlEntity(), null);
+        });
+      }
+    } catch (FunctionInvocationTargetException ignore) {
+      result = ResultBuilder.createGemFireErrorResult(CliStrings.format(
+          CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN, LuceneCliStrings.LUCENE_DESTROY_INDEX));
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (IllegalArgumentException e) {
+      result = ResultBuilder.createInfoResult(e.getMessage());
+    } catch (Throwable t) {
+      SystemFailure.checkFailure();
+      getCache().getLogger().warning(LuceneCliStrings.LUCENE_DESTROY_INDEX__EXCEPTION_MESSAGE, t);
+      result = ResultBuilder.createGemFireErrorResult(t.getMessage());
+    }
+    return result;
+  }
+
   private Result displayResults(int pageSize, boolean keysOnly) throws Exception {
     if (searchResults.size() == 0) {
       return ResultBuilder
@@ -428,25 +494,30 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
 
   protected ResultCollector<?, ?> executeFunctionOnGroups(FunctionAdapter function, String[] groups,
       final LuceneIndexInfo indexInfo) throws IllegalArgumentException, CommandResultException {
-    final Set<DistributedMember> targetMembers;
+    ResultCollector<?, ?> results = null;
     if (function != createIndexFunction) {
-      targetMembers =
-          CliUtil.getMembersForeRegionViaFunction(getCache(), indexInfo.getRegionPath(), true);
-      if (targetMembers.isEmpty()) {
-        throw new IllegalArgumentException("Region not found.");
-      }
+      results = executeFunction(function, indexInfo, true);
     } else {
-      targetMembers = CliUtil.findMembersOrThrow(groups, null);
+      Set<DistributedMember> targetMembers = CliUtil.findMembersOrThrow(groups, null);
+      results = CliUtil.executeFunction(function, indexInfo, targetMembers);
     }
-    return CliUtil.executeFunction(function, indexInfo, targetMembers);
+    return results;
   }
 
   protected ResultCollector<?, ?> executeSearch(final LuceneQueryInfo queryInfo) throws Exception {
-    final Set<DistributedMember> targetMembers =
-        CliUtil.getMembersForeRegionViaFunction(getCache(), queryInfo.getRegionPath(), false);
-    if (targetMembers.isEmpty())
-      throw new IllegalArgumentException("Region not found.");
-    return CliUtil.executeFunction(searchIndexFunction, queryInfo, targetMembers);
+    return executeFunction(searchIndexFunction, queryInfo, false);
+  }
+
+  protected ResultCollector<?, ?> executeFunction(Function function,
+      LuceneFunctionSerializable functionArguments, boolean returnAllMembers) {
+    Set<DistributedMember> targetMembers = CliUtil.getMembersForeRegionViaFunction(getCache(),
+        functionArguments.getRegionPath(), returnAllMembers);
+    if (targetMembers.isEmpty()) {
+      throw new IllegalArgumentException(CliStrings.format(
+          LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__COULDNOT_FIND_MEMBERS_FOR_REGION_0,
+          new Object[] {functionArguments.getRegionPath()}));
+    }
+    return CliUtil.executeFunction(function, functionArguments, targetMembers);
   }
 
   @CliAvailabilityIndicator({LuceneCliStrings.LUCENE_SEARCH_INDEX,

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexDetails.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexDetails.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexDetails.java
index 02a19a1..b2b0466 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexDetails.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexDetails.java
@@ -29,10 +29,9 @@ import org.apache.geode.cache.lucene.internal.LuceneIndexStats;
 
 import org.apache.lucene.analysis.Analyzer;
 
-public class LuceneIndexDetails implements Comparable<LuceneIndexDetails>, Serializable {
+public class LuceneIndexDetails extends LuceneFunctionSerializable
+    implements Comparable<LuceneIndexDetails> {
   private static final long serialVersionUID = 1L;
-  private final String indexName;
-  private final String regionPath;
   private final String serverName;
   private final String[] searchableFieldNames;
   private Map<String, String> fieldAnalyzers = null;
@@ -42,8 +41,7 @@ public class LuceneIndexDetails implements Comparable<LuceneIndexDetails>, Seria
   public LuceneIndexDetails(final String indexName, final String regionPath,
       final String[] searchableFieldNames, final Map<String, Analyzer> fieldAnalyzers,
       LuceneIndexStats indexStats, boolean initialized, final String serverName) {
-    this.indexName = indexName;
-    this.regionPath = regionPath;
+    super(indexName, regionPath);
     this.serverName = serverName;
     this.searchableFieldNames = searchableFieldNames;
     this.fieldAnalyzers = getFieldAnalyzerStrings(fieldAnalyzers);
@@ -141,14 +139,6 @@ public class LuceneIndexDetails implements Comparable<LuceneIndexDetails>, Seria
     return initialized;
   }
 
-  public String getIndexName() {
-    return indexName;
-  }
-
-  public String getRegionPath() {
-    return regionPath;
-  }
-
   private static <T extends Comparable<T>> int compare(final T obj1, final T obj2) {
     return (obj1 == null && obj2 == null ? 0
         : (obj1 == null ? 1 : (obj2 == null ? -1 : obj1.compareTo(obj2))));

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexInfo.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexInfo.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexInfo.java
index b714ff1..41b066e 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexInfo.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexInfo.java
@@ -24,18 +24,15 @@ import org.apache.geode.cache.lucene.internal.LuceneIndexImpl;
 
 import org.apache.lucene.analysis.Analyzer;
 
-public class LuceneIndexInfo implements Serializable {
+public class LuceneIndexInfo extends LuceneFunctionSerializable {
   private static final long serialVersionUID = 1L;
 
-  private final String indexName;
-  private final String regionPath;
   private final String[] searchableFieldNames;
   private final String[] fieldAnalyzers;
 
   public LuceneIndexInfo(final String indexName, final String regionPath,
       final String[] searchableFieldNames, String[] fieldAnalyzers) {
-    this.indexName = indexName;
-    this.regionPath = regionPath;
+    super(indexName, regionPath);
     this.searchableFieldNames = searchableFieldNames;
     this.fieldAnalyzers = fieldAnalyzers;
   }
@@ -44,14 +41,6 @@ public class LuceneIndexInfo implements Serializable {
     this(indexName, regionPath, null, null);
   }
 
-  public String getIndexName() {
-    return indexName;
-  }
-
-  public String getRegionPath() {
-    return regionPath;
-  }
-
   public String[] getSearchableFieldNames() {
     return searchableFieldNames;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneQueryInfo.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneQueryInfo.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneQueryInfo.java
index 8d34e2e..e57badb 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneQueryInfo.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneQueryInfo.java
@@ -19,10 +19,8 @@ import java.io.Serializable;
 
 import org.apache.geode.cache.lucene.LuceneQueryFactory;
 
-public class LuceneQueryInfo implements Serializable {
+public class LuceneQueryInfo extends LuceneFunctionSerializable {
   private static final long serialVersionUID = 1L;
-  private String indexName;
-  private String regionPath;
   private String queryString;
   private String defaultField;
   private int limit;
@@ -30,22 +28,13 @@ public class LuceneQueryInfo implements Serializable {
 
   public LuceneQueryInfo(final String indexName, final String regionPath, final String queryString,
       final String defaultField, final int limit, final boolean keysOnly) {
-    this.indexName = indexName;
-    this.regionPath = regionPath;
+    super(indexName, regionPath);
     this.queryString = queryString;
     this.defaultField = defaultField;
     this.limit = limit;
     this.keysOnly = keysOnly;
   }
 
-  public String getIndexName() {
-    return indexName;
-  }
-
-  public String getRegionPath() {
-    return regionPath;
-  }
-
   public String getQueryString() {
     return queryString;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunction.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1535637
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunction.java
@@ -0,0 +1,57 @@
+/*
+ * 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.functions;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.lucene.LuceneService;
+import org.apache.geode.cache.lucene.LuceneServiceProvider;
+import org.apache.geode.cache.lucene.internal.cli.LuceneIndexInfo;
+import org.apache.geode.internal.InternalEntity;
+import org.apache.geode.internal.cache.xmlcache.CacheXml;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
+import org.apache.geode.management.internal.configuration.domain.XmlEntity;
+
+public class LuceneDestroyIndexFunction implements Function, InternalEntity {
+
+  public void execute(final FunctionContext context) {
+    String memberId = getCache().getDistributedSystem().getDistributedMember().getId();
+    try {
+      LuceneIndexInfo indexInfo = (LuceneIndexInfo) context.getArguments();
+      String indexName = indexInfo.getIndexName();
+      String regionPath = indexInfo.getRegionPath();
+      LuceneService service = LuceneServiceProvider.get(getCache());
+      if (indexName == null) {
+        service.destroyIndexes(regionPath);
+      } else {
+        service.destroyIndex(indexName, regionPath);
+      }
+      context.getResultSender()
+          .lastResult(new CliFunctionResult(memberId, getXmlEntity(regionPath)));
+    } catch (Exception e) {
+      context.getResultSender().lastResult(new CliFunctionResult(memberId, e, e.getMessage()));
+    }
+  }
+
+  protected XmlEntity getXmlEntity(String regionPath) {
+    return new XmlEntity(CacheXml.REGION, "name", regionPath);
+  }
+
+  protected Cache getCache() {
+    return CacheFactory.getAnyInstance();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..6260075
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
@@ -0,0 +1,247 @@
+/*
+ * 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;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
+import org.apache.geode.cache.lucene.test.TestObject;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.test.dunit.AsyncInvocation;
+import org.apache.geode.test.dunit.SerializableRunnableIF;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.awaitility.Awaitility;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.util.concurrent.TimeUnit;
+
+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.internal.Assert.fail;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+@Category(DistributedTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class LuceneIndexDestroyDUnitTest extends LuceneDUnitTest {
+
+  private volatile boolean STOP_PUTS = false;
+
+  private final Object[] parametersForIndexDestroys() {
+    String[] destroyDataRegionParameters = {"true", "false"};
+    RegionTestableType[] regionTestTypes = getListOfRegionTestTypes();
+    return parameterCombiner(destroyDataRegionParameters, regionTestTypes);
+  }
+
+  @Test
+  @Parameters(method = "parametersForIndexDestroys")
+  public void verifyDestroySingleIndex(boolean destroyDataRegion, RegionTestableType regionType) {
+    // Create index and region
+    dataStore1.invoke(() -> initDataStore(createIndex(), regionType));
+    dataStore2.invoke(() -> initDataStore(createIndex(), regionType));
+
+    // Verify index created
+    dataStore1.invoke(() -> verifyIndexCreated());
+    dataStore2.invoke(() -> verifyIndexCreated());
+
+    // Attempt to destroy data region (should fail)
+    if (destroyDataRegion) {
+      dataStore1.invoke(() -> destroyDataRegion(false));
+    }
+
+    // Destroy index (only needs to be done on one member)
+    dataStore1.invoke(() -> destroyIndex());
+
+    // Verify index destroyed
+    dataStore1.invoke(() -> verifyIndexDestroyed());
+    dataStore2.invoke(() -> verifyIndexDestroyed());
+
+    // Attempt to destroy data region (should succeed)
+    if (destroyDataRegion) {
+      dataStore1.invoke(() -> destroyDataRegion(true));
+    }
+  }
+
+  @Test
+  @Parameters(method = "parametersForIndexDestroys")
+  public void verifyDestroyAllIndexes(boolean destroyDataRegion, RegionTestableType regionType) {
+    // Create indexes and region
+    dataStore1.invoke(() -> initDataStore(createIndexes(), regionType));
+    dataStore2.invoke(() -> initDataStore(createIndexes(), regionType));
+
+    // Verify indexes created
+    dataStore1.invoke(() -> verifyIndexesCreated());
+    dataStore2.invoke(() -> verifyIndexesCreated());
+
+    // Attempt to destroy data region (should fail)
+    if (destroyDataRegion) {
+      dataStore1.invoke(() -> destroyDataRegion(false));
+    }
+
+    // Destroy indexes (only needs to be done on one member)
+    dataStore1.invoke(() -> destroyIndexes());
+
+    // Verify indexes destroyed
+    dataStore1.invoke(() -> verifyIndexesDestroyed());
+    dataStore2.invoke(() -> verifyIndexesDestroyed());
+
+    // Attempt to destroy data region (should succeed)
+    if (destroyDataRegion) {
+      dataStore1.invoke(() -> destroyDataRegion(true));
+    }
+  }
+
+  @Ignore
+  // Destroying an index while puts are occurring currently fails with a
+  // GatewaySenderConfigurationException.
+  @Parameters(method = "getListOfServerRegionTestTypes")
+  public void verifyDestroySingleIndexWhileDoingPuts(RegionTestableType regionType)
+      throws Exception {
+    // Create index and region
+    dataStore1.invoke(() -> initDataStore(createIndex(), regionType));
+    dataStore2.invoke(() -> initDataStore(createIndex(), regionType));
+
+    // Verify index created
+    dataStore1.invoke(() -> verifyIndexCreated());
+    dataStore2.invoke(() -> verifyIndexCreated());
+
+    // Start puts
+    AsyncInvocation putter = dataStore1.invokeAsync(() -> doPuts());
+
+    // Wait until puts have started
+    dataStore1.invoke(() -> waitUntilPutsHaveStarted());
+
+    // Destroy index (only needs to be done on one member)
+    dataStore1.invoke(() -> destroyIndex());
+
+    // Verify index destroyed
+    dataStore1.invoke(() -> verifyIndexDestroyed());
+    dataStore2.invoke(() -> verifyIndexDestroyed());
+
+    // End puts
+    dataStore1.invoke(() -> stopPuts());
+    putter.join();
+  }
+
+
+  private SerializableRunnableIF createIndex() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+    };
+  }
+
+  private SerializableRunnableIF createIndexes() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME + "0", REGION_NAME, "text");
+      luceneService.createIndex(INDEX_NAME + "1", REGION_NAME, "text");
+    };
+  }
+
+  private void verifyIndexCreated() {
+    LuceneService luceneService = LuceneServiceProvider.get(getCache());
+    assertNotNull(luceneService.getIndex(INDEX_NAME, REGION_NAME));
+  }
+
+  private void verifyIndexesCreated() {
+    LuceneService luceneService = LuceneServiceProvider.get(getCache());
+    assertNotNull(luceneService.getIndex(INDEX_NAME + "0", REGION_NAME));
+    assertNotNull(luceneService.getIndex(INDEX_NAME + "1", REGION_NAME));
+  }
+
+  private void doPuts() throws Exception {
+    Region region = getCache().getRegion(REGION_NAME);
+    int i = 0;
+    while (!STOP_PUTS) {
+      region.put(i++, new TestObject());
+      // Thread.sleep(50);
+    }
+  }
+
+  private void stopPuts() {
+    STOP_PUTS = true;
+  }
+
+  private void waitUntilPutsHaveStarted() {
+    Awaitility.waitAtMost(30, TimeUnit.SECONDS)
+        .until(() -> getCache().getRegion(REGION_NAME).size() > 0);
+  }
+
+  private void destroyDataRegion(boolean shouldSucceed) {
+    Region region = getCache().getRegion(REGION_NAME);
+    assertNotNull(region);
+    try {
+      region.destroyRegion();
+      if (!shouldSucceed) {
+        fail("should not have been able to destroy data region named " + region.getFullPath());
+      }
+    } catch (IllegalStateException e) {
+      if (shouldSucceed) {
+        fail(e);
+      }
+    }
+  }
+
+  private void destroyIndex() {
+    LuceneService luceneService = LuceneServiceProvider.get(getCache());
+    luceneService.destroyIndex(INDEX_NAME, REGION_NAME);
+  }
+
+  private void destroyIndexes() {
+    LuceneService luceneService = LuceneServiceProvider.get(getCache());
+    luceneService.destroyIndexes(REGION_NAME);
+  }
+
+  private void verifyIndexDestroyed() {
+    verifyIndexDestroyed(INDEX_NAME);
+  }
+
+  private void verifyIndexesDestroyed() {
+    verifyIndexDestroyed(INDEX_NAME + "0");
+    verifyIndexDestroyed(INDEX_NAME + "1");
+  }
+
+  private void verifyIndexDestroyed(String indexName) {
+    LuceneService luceneService = LuceneServiceProvider.get(getCache());
+
+    // Verify the index itself no longer exists
+    assertNull(luceneService.getIndex(indexName, REGION_NAME));
+
+    // Verify the underlying files region no longer exists
+    String filesRegionName = LuceneServiceImpl.getUniqueIndexRegionName(indexName, REGION_NAME,
+        LuceneIndexForPartitionedRegion.FILES_REGION_SUFFIX);
+    assertNull(getCache().getRegion(filesRegionName));
+
+    // Verify the underlying chunks region no longer exists
+    String chunksRegionName = LuceneServiceImpl.getUniqueIndexRegionName(indexName, REGION_NAME,
+        LuceneIndexForPartitionedRegion.CHUNKS_REGION_SUFFIX);
+    assertNull(getCache().getRegion(chunksRegionName));
+
+    // Verify the underlying AsyncEventQueue no longer exists
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, REGION_NAME);
+    assertNull(getCache().getAsyncEventQueue(aeqId));
+
+    // Verify the data region extension no longer exists
+    LocalRegion region = (LocalRegion) getCache().getRegion(REGION_NAME);
+    assertFalse(region.getExtensionPoint().getExtensions().iterator().hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
----------------------------------------------------------------------
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 49aa057..efc11ab 100755
--- 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
@@ -28,6 +28,7 @@ import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.management.cli.Result.Status;
 import org.apache.geode.management.internal.cli.CommandManager;
 import org.apache.geode.management.internal.cli.commands.CliCommandTestBase;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.TabularResultData;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
@@ -349,7 +350,7 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, "notAnIndex");
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
     String resultAsString = executeCommandAndLogResult(csb);
-    assertTrue(resultAsString.contains("Region not found"));
+    assertTrue(resultAsString.contains(getRegionNotFoundErrorMessage(REGION_NAME)));
   }
 
   @Test
@@ -498,7 +499,7 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__DEFAULT_FIELD, "field2");
 
     String resultAsString = executeCommandAndLogResult(csb);
-    assertTrue(resultAsString.contains("Region not found"));
+    assertTrue(resultAsString.contains(getRegionNotFoundErrorMessage(REGION_NAME)));
   }
 
   @Test
@@ -552,6 +553,45 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
 
   }
 
+  @Test
+  public void destroySingleIndexOnRegion() throws Exception {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createIndex(vm1);
+    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    String resultAsString = executeCommandAndLogResult(csb);
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FOR_REGION_1,
+        new Object[] {INDEX_NAME, REGION_NAME});
+    assertTrue(resultAsString.contains(expectedStatus));
+  }
+
+  @Test
+  public void destroyAllIndexesOnRegion() throws Exception {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createIndex(vm1);
+    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    String resultAsString = executeCommandAndLogResult(csb);
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FOR_REGION_0,
+        new Object[] {REGION_NAME});
+    assertTrue(resultAsString.contains(expectedStatus));
+  }
+
+  @Test
+  public void destroyIndexWithoutRegionShouldReturnError() throws Exception {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    createIndexWithoutRegion(vm1);
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    String resultAsString = executeCommandAndLogResult(csb);
+    assertTrue(resultAsString.contains(getRegionNotFoundErrorMessage(REGION_NAME)));
+  }
+
   private void createRegion() {
     getCache().createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
   }
@@ -628,6 +668,12 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     });
   }
 
+  private String getRegionNotFoundErrorMessage(String regionPath) {
+    return CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__COULDNOT_FIND_MEMBERS_FOR_REGION_0,
+        new Object[] {regionPath});
+  }
+
   protected class TestObject implements Serializable {
     private String field1;
     private String field2;

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsJUnitTest.java
----------------------------------------------------------------------
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 6ddef9e..9e8d7a9 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
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
@@ -41,6 +42,7 @@ import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.lucene.internal.LuceneIndexStats;
 import org.apache.geode.cache.lucene.internal.cli.functions.LuceneCreateIndexFunction;
 import org.apache.geode.cache.lucene.internal.cli.functions.LuceneDescribeIndexFunction;
+import org.apache.geode.cache.lucene.internal.cli.functions.LuceneDestroyIndexFunction;
 import org.apache.geode.cache.lucene.internal.cli.functions.LuceneListIndexFunction;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
@@ -401,6 +403,57 @@ public class LuceneIndexCommandsJUnitTest {
     assertEquals(queryResults.size(), data.retrieveAllValues("key").size());
   }
 
+  @Test
+  public void testDestroySingleIndexOnRegion() throws Exception {
+    LuceneIndexCommands commands = createTestLuceneIndexCommandsForDestroyIndex();
+    String indexName = "index";
+    String regionPath = "regionPath";
+    CommandResult result = (CommandResult) commands.destroyIndex(indexName, regionPath);
+    String expectedMember = "member";
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEX_0_FOR_REGION_1,
+        new Object[] {indexName, regionPath});
+    verifyDestroyIndexCommandResult(result, expectedMember, expectedStatus);
+  }
+
+  @Test
+  public void testDestroyAllIndexesOnRegion() throws Exception {
+    LuceneIndexCommands commands = createTestLuceneIndexCommandsForDestroyIndex();
+    String indexName = null;
+    String regionPath = "regionPath";
+    CommandResult result = (CommandResult) commands.destroyIndex(indexName, regionPath);
+    String expectedMember = "member";
+    String expectedStatus = CliStrings.format(
+        LuceneCliStrings.LUCENE_DESTROY_INDEX__MSG__SUCCESSFULLY_DESTROYED_INDEXES_FOR_REGION_0,
+        new Object[] {regionPath});
+    verifyDestroyIndexCommandResult(result, expectedMember, expectedStatus);
+  }
+
+  private LuceneIndexCommands createTestLuceneIndexCommandsForDestroyIndex() {
+    final Cache mockCache = mock(Cache.class);
+    final ResultCollector mockResultCollector = mock(ResultCollector.class);
+    final LuceneIndexCommands commands = spy(createIndexCommands(mockCache, null));
+
+    final List<CliFunctionResult> cliFunctionResults = new ArrayList<>();
+    cliFunctionResults.add(new CliFunctionResult("member", true, "Index Destroyed"));
+
+    doReturn(mockResultCollector).when(commands).executeFunction(
+        isA(LuceneDestroyIndexFunction.class), any(LuceneIndexInfo.class), eq(false));
+    doReturn(cliFunctionResults).when(mockResultCollector).getResult();
+    return commands;
+  }
+
+  private void verifyDestroyIndexCommandResult(CommandResult result, String expectedMember,
+      String expectedStatus) {
+    assertEquals(Status.OK, result.getStatus());
+    TabularResultData data = (TabularResultData) result.getResultData();
+    List<String> members = data.retrieveAllValues("Member");
+    List<String> status = data.retrieveAllValues("Status");
+    assertTrue(members.size() == 1);
+    assertEquals(expectedMember, members.get(0));
+    assertEquals(expectedStatus, status.get(0));
+  }
+
   private String getPage(final LuceneSearchResults[] expectedResults, int[] indexList) {
     final TabularResultData data = ResultBuilder.createTabularResultData();
     for (int i : indexList) {

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f86f4a1
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.functions;
+
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.cache.lucene.internal.InternalLuceneService;
+import org.apache.geode.cache.lucene.internal.cli.LuceneIndexInfo;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
+import org.apache.geode.management.internal.configuration.domain.XmlEntity;
+import org.apache.geode.test.fake.Fakes;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+@Category(UnitTest.class)
+public class LuceneDestroyIndexFunctionJUnitTest {
+
+  private InternalLuceneService service;
+  private GemFireCacheImpl cache;
+  private String member;
+  private FunctionContext context;
+  private ResultSender resultSender;
+
+  @Before
+  public void prepare() {
+    this.cache = Fakes.cache();
+    this.member = Fakes.distributedSystem().getDistributedMember().getId();
+    this.service = mock(InternalLuceneService.class);
+    when(this.cache.getService(InternalLuceneService.class)).thenReturn(this.service);
+    this.context = mock(FunctionContext.class);
+    this.resultSender = mock(ResultSender.class);
+    when(this.context.getResultSender()).thenReturn(this.resultSender);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testExecuteWithRegionAndIndex() throws Throwable {
+    LuceneIndexInfo indexInfo = new LuceneIndexInfo("index1", "/region1");
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    doReturn(mock(XmlEntity.class)).when(function).getXmlEntity(any());
+    function.execute(this.context);
+    verify(this.service).destroyIndex(eq("index1"), eq("/region1"));
+    verify(this.service, never()).destroyIndexes(eq("/region1"));
+    ArgumentCaptor<CliFunctionResult> resultCaptor =
+        ArgumentCaptor.forClass(CliFunctionResult.class);
+    verify(resultSender).lastResult(resultCaptor.capture());
+    CliFunctionResult result = resultCaptor.getValue();
+    assertEquals(this.member, result.getMemberIdOrName());
+    assertEquals(true, result.isSuccessful());
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testExecuteWithRegion() throws Throwable {
+    LuceneIndexInfo indexInfo = new LuceneIndexInfo(null, "/region1");
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    doReturn(mock(XmlEntity.class)).when(function).getXmlEntity(any());
+    function.execute(this.context);
+    verify(this.service).destroyIndexes(eq("/region1"));
+    verify(this.service, never()).destroyIndex(any(), eq("/region1"));
+    ArgumentCaptor<CliFunctionResult> resultCaptor =
+        ArgumentCaptor.forClass(CliFunctionResult.class);
+    verify(resultSender).lastResult(resultCaptor.capture());
+    CliFunctionResult result = resultCaptor.getValue();
+    assertEquals(this.member, result.getMemberIdOrName());
+    assertEquals(true, result.isSuccessful());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/11521a82/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
index 875422c..1a344db 100755
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
@@ -17,10 +17,7 @@ package org.apache.geode.cache.lucene.internal.configuration;
 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.GROUPS;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
+import static org.junit.Assert.*;
 
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.LuceneIndex;
@@ -28,9 +25,14 @@ import org.apache.geode.cache.lucene.LuceneService;
 import org.apache.geode.cache.lucene.LuceneServiceProvider;
 import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
 import org.apache.geode.cache.lucene.internal.cli.LuceneIndexCommands;
+import org.apache.geode.cache.lucene.internal.xml.LuceneXmlConstants;
+import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.management.internal.cli.CommandManager;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.management.internal.configuration.domain.Configuration;
+import org.apache.geode.test.dunit.SerializableRunnableIF;
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
 import org.apache.geode.test.dunit.rules.Locator;
 import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
@@ -231,6 +233,78 @@ public class LuceneClusterConfigurationDUnitTest {
     });
   }
 
+  @Test
+  public void verifyClusterConfigurationAfterDestroyIndex() throws Exception {
+    Member vm1 = startNodeUsingClusterConfiguration(1, false);
+
+    // Connect Gfsh to locator.
+    gfshConnector.connectAndVerify(locator);
+
+    // Create and add indexes
+    createAndAddIndexes();
+
+    // Destroy one index
+    destroyLuceneIndexUsingGfsh(INDEX_NAME + "0");
+
+    // Destroy other index
+    destroyLuceneIndexUsingGfsh(INDEX_NAME + "1");
+
+    // Verify cluster configuration no longer contains any indexes
+    locator.invoke(verifyClusterConfiguration(false));
+  }
+
+  @Test
+  public void verifyClusterConfigurationAfterDestroyIndexes() throws Exception {
+    Member vm1 = startNodeUsingClusterConfiguration(1, false);
+
+    // Connect Gfsh to locator.
+    gfshConnector.connectAndVerify(locator);
+
+    // Create and add indexes
+    createAndAddIndexes();
+
+    // Destroy all indexes
+    destroyLuceneIndexUsingGfsh(null);
+
+    // Verify cluster configuration no longer contains indexes
+    locator.invoke(verifyClusterConfiguration(false));
+  }
+
+  private void createAndAddIndexes() throws Exception {
+    // Create lucene index.
+    createLuceneIndexUsingGfsh(INDEX_NAME + "0", false);
+
+    // Create another lucene index.
+    createLuceneIndexUsingGfsh(INDEX_NAME + "1", false);
+
+    // Create region
+    createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, null);
+
+    // Verify cluster configuration contains the indexes
+    locator.invoke(verifyClusterConfiguration(true));
+  }
+
+  private SerializableRunnableIF verifyClusterConfiguration(boolean verifyIndexesExist) {
+    return () -> {
+      InternalLocator internalLocator = LocatorServerStartupRule.locatorStarter.locator;
+      ClusterConfigurationService sc = internalLocator.getSharedConfiguration();
+      Configuration config = sc.getConfiguration(ClusterConfigurationService.CLUSTER_CONFIG);
+      String xmlContent = config.getCacheXmlContent();
+      String luceneIndex0Config = "<" + LuceneXmlConstants.PREFIX + ":" + LuceneXmlConstants.INDEX
+          + " xmlns:lucene=\"" + LuceneXmlConstants.NAMESPACE + "\" " + LuceneXmlConstants.NAME
+          + "=\"" + INDEX_NAME + "0" + "\">";
+      String luceneIndex1Config = "<" + LuceneXmlConstants.PREFIX + ":" + LuceneXmlConstants.INDEX
+          + " xmlns:lucene=\"" + LuceneXmlConstants.NAMESPACE + "\" " + LuceneXmlConstants.NAME
+          + "=\"" + INDEX_NAME + "1" + "\">";
+      if (verifyIndexesExist) {
+        assertTrue(xmlContent.contains(luceneIndex0Config));
+        assertTrue(xmlContent.contains(luceneIndex1Config));
+      } else {
+        assertFalse(xmlContent.contains(luceneIndex0Config));
+        assertFalse(xmlContent.contains(luceneIndex1Config));
+      }
+    };
+  }
 
   private Member startNodeUsingClusterConfiguration(int vmIndex, boolean addGroup)
       throws Exception {
@@ -242,10 +316,14 @@ public class LuceneClusterConfigurationDUnitTest {
   }
 
   private void createLuceneIndexUsingGfsh(boolean addGroup) throws Exception {
+    createLuceneIndexUsingGfsh(INDEX_NAME, addGroup);
+  }
+
+  private void createLuceneIndexUsingGfsh(String indexName, boolean addGroup) throws Exception {
     // Execute Gfsh command to create lucene index.
     CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
-    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, indexName);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
     if (addGroup) {
       csb.addOption(LuceneCliStrings.LUCENE_CREATE_INDEX__GROUP, groupName);
@@ -273,6 +351,17 @@ public class LuceneClusterConfigurationDUnitTest {
     gfshConnector.executeAndVerifyCommand(csb.toString());
   }
 
+  private void destroyLuceneIndexUsingGfsh(String indexName) throws Exception {
+    // Execute Gfsh command to destroy lucene index.
+    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
+    if (indexName != null) {
+      csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, indexName);
+    }
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    gfshConnector.executeAndVerifyCommand(csb.toString());
+  }
+
   private void createRegionUsingGfsh(String regionName, RegionShortcut regionShortCut, String group)
       throws Exception {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_REGION);