You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by nn...@apache.org on 2018/08/03 04:02:49 UTC

[geode] branch develop updated: GEODE-5055: Handle index in progress for old clients (#1961)

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

nnag pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 06124e0  GEODE-5055: Handle index in progress for old clients (#1961)
06124e0 is described below

commit 06124e0c4191a90fa384a1368fc59a7d8dfdc0c8
Author: Nabarun Nag <na...@users.noreply.github.com>
AuthorDate: Thu Aug 2 21:02:44 2018 -0700

    GEODE-5055: Handle index in progress for old clients (#1961)
    
    * If the Lucene query function is executed by an old client (< 1.6.0) on a new server, it will wait for the index to be created.
                * Server wont return a LuceneIndexCreationInProgressException back to the old client resulting in a ClassNotFoundException.
    	    * LuceneIndexCreationInProgressException is wrapped in a FunctionException and sent to the caller function
    	    * The caller unwraps and send the LuceneQueryException back to the user.
---
 .../PartitionedRegionFunctionResultSender.java     |  9 ++++
 .../geode/internal/i18n/LocalizedStrings.java      |  6 +++
 .../geode/cache/lucene/LuceneQueriesDUnitTest.java | 16 ++++++
 .../lucene/LuceneQueriesReindexDUnitTest.java      | 53 ++++++++++++++++++
 .../apache/geode/cache/lucene/LuceneService.java   | 16 ++++++
 .../cache/lucene/internal/LuceneQueryImpl.java     |  4 +-
 .../cache/lucene/internal/LuceneServiceImpl.java   | 48 +++++++++++++++++
 .../distributed/IndexingInProgressFunction.java    | 63 ++++++++++++++++++++++
 .../internal/distributed/LuceneQueryFunction.java  | 33 +++++++++---
 .../sanctioned-geode-lucene-serializables.txt      |  5 +-
 10 files changed, 240 insertions(+), 13 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/PartitionedRegionFunctionResultSender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/PartitionedRegionFunctionResultSender.java
index 4f4dcc5..3f5f04a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/PartitionedRegionFunctionResultSender.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/PartitionedRegionFunctionResultSender.java
@@ -24,6 +24,7 @@ import org.apache.geode.cache.execute.FunctionException;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.ForceReattemptException;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.partitioned.PartitionedRegionFunctionStreamingMessage;
@@ -74,6 +75,14 @@ public class PartitionedRegionFunctionResultSender implements InternalResultSend
 
   private BucketMovedException bme;
 
+
+  public Version getClientVersion() {
+    if (serverSender != null && serverSender.sc != null) { // is a client-server connection
+      return serverSender.sc.getClientVersion();
+    }
+    return null;
+  }
+
   /**
    * Have to combine next two constructor in one and make a new class which will send Results back.
    *
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 768f228..0ffcc10 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
@@ -7736,6 +7736,12 @@ public class LocalizedStrings {
 
   public static final StringId LuceneIndexCreation_INDEX_WAS_DESTROYED_WHILE_BEING_CREATED =
       new StringId(6675, "Lucene index {0} on region {1} was destroyed while being created");
+
+  public static final StringId LuceneIndexingInProgress_CANNOT_BE_DETERMINED_BECAUSE_OF_VERSION_MISMATCH =
+      new StringId(6676,
+          "Lucene indexing in progress status cannot be determined if all members hosting the user data region : {0}, are not above Apache Geode 1.6.0 version ");
+  public static final StringId LuceneQueryException_INDEX_NOT_AVAILABLE_CURRENTLY_INDEXING =
+      new StringId(6677, "Lucene Index is not available, currently indexing");
   /** Testing strings, messageId 90000-99999 **/
 
   /**
diff --git a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java
index e4f91d3..e9dce44 100644
--- a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java
+++ b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesDUnitTest.java
@@ -23,11 +23,13 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.TermQuery;
+import org.awaitility.Awaitility;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -100,6 +102,20 @@ public class LuceneQueriesDUnitTest extends LuceneQueriesAccessorBase {
 
   @Test
   @Parameters(method = "getListOfRegionTestTypes")
+  public void afterLuceneIndexAndRegionIsCreatedShouldBeAbleToGetIndexingStatus(
+      RegionTestableType regionTestType) throws Exception {
+    createRegionAndIndexForAllDataStores(regionTestType, createIndex);
+    putDataInRegion(accessor);
+    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
+    assertTrue(waitForFlushBeforeExecuteTextSearch(dataStore1, 60000));
+    accessor.invoke(() -> Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> assertFalse(
+        LuceneServiceProvider.get(getCache()).isIndexingInProgress(INDEX_NAME, REGION_NAME))));
+    executeTextSearch(accessor);
+
+  }
+
+  @Test
+  @Parameters(method = "getListOfRegionTestTypes")
   public void returnCorrectResultsFromStringQueryWithDefaultAnalyzer(
       RegionTestableType regionTestType) throws Exception {
     createRegionAndIndexForAllDataStores(regionTestType, createIndex);
diff --git a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesReindexDUnitTest.java b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesReindexDUnitTest.java
index db3e5b9..f7cabe6 100644
--- a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesReindexDUnitTest.java
+++ b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneQueriesReindexDUnitTest.java
@@ -14,13 +14,18 @@
  */
 package org.apache.geode.cache.lucene;
 
+import static org.apache.geode.cache.lucene.LuceneDUnitTest.RegionTestableType.PARTITION_WITH_CLIENT;
 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.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.util.concurrent.TimeUnit;
+
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
+import org.awaitility.Awaitility;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -97,6 +102,54 @@ public class LuceneQueriesReindexDUnitTest extends LuceneQueriesAccessorBase {
 
   @Test
   @Parameters(method = "getListOfRegionTestTypes")
+  public void afterReindexingWeShouldBeAbleToGetTheStatusOfIndexingProgress(
+      RegionTestableType regionTestType) throws Exception {
+    dataStore1.invoke(() -> {
+      regionTestType.createDataStore(getCache(), REGION_NAME);
+    });
+    dataStore2.invoke(() -> {
+      regionTestType.createDataStore(getCache(), REGION_NAME);
+    });
+    accessor.invoke(() -> {
+      regionTestType.createAccessor(getCache(), REGION_NAME);
+    });
+
+    putDataInRegion(accessor);
+
+    // re-index stored data
+    AsyncInvocation ai1 = dataStore1.invokeAsync(() -> {
+      createIndex("text");
+    });
+
+    AsyncInvocation ai2 = dataStore2.invokeAsync(() -> {
+      createIndex("text");
+    });
+
+    AsyncInvocation ai3 = accessor.invokeAsync(() -> {
+      if (regionTestType != PARTITION_WITH_CLIENT) {
+        createIndex("text");
+      }
+    });
+
+    ai1.join();
+    ai2.join();
+    ai3.join();
+
+    ai1.checkException();
+    ai2.checkException();
+    ai3.checkException();
+
+    waitForFlushBeforeExecuteTextSearch(accessor, 60000);
+
+    accessor.invoke(() -> Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> assertFalse(
+        LuceneServiceProvider.get(getCache()).isIndexingInProgress(INDEX_NAME, REGION_NAME))));
+    executeTextSearch(accessor);
+  }
+
+
+
+  @Test
+  @Parameters(method = "getListOfRegionTestTypes")
   public void dropAndRecreateIndexWithDifferentFieldsShouldFail(RegionTestableType regionTestType)
       throws Exception {
     SerializableRunnableIF createIndex = () -> {
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 23f2c28..347199d 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
@@ -23,6 +23,7 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.GemFireCache;
+import org.apache.geode.cache.lucene.internal.LuceneIndexCreationInProgressException;
 
 /**
  *
@@ -172,4 +173,19 @@ public interface LuceneService {
    */
   boolean waitUntilFlushed(String indexName, String regionPath, long timeout, TimeUnit unit)
       throws InterruptedException;
+
+  /**
+   * Returns if the indexing process is in progress
+   *
+   * Before executing a lucene query, it can be checked if the indexing operation is in progress.
+   * Queries executed during the indexing process will get a
+   * {@link LuceneIndexCreationInProgressException}
+   *
+   * @param indexName index name
+   *
+   * @param regionPath region name
+   *
+   * @return true if the indexing operation is in progress otherwise false.
+   */
+  boolean isIndexingInProgress(String indexName, String regionPath);
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneQueryImpl.java
index 1026a83..d010c48 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneQueryImpl.java
@@ -122,7 +122,7 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
       entries = rc.getResult();
     } catch (FunctionException e) {
       if (e.getCause() instanceof LuceneQueryException) {
-        throw new LuceneQueryException(e);
+        throw (LuceneQueryException) e.getCause();
       } else if (e.getCause() instanceof TransactionException) {
         // When run from client with single hop disabled
         throw new LuceneQueryException(LUCENE_QUERY_CANNOT_BE_EXECUTED_WITHIN_A_TRANSACTION);
@@ -133,8 +133,6 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
     } catch (TransactionException e) {
       // When function execution is run from server
       throw new LuceneQueryException(LUCENE_QUERY_CANNOT_BE_EXECUTED_WITHIN_A_TRANSACTION);
-    } catch (LuceneIndexCreationInProgressException e) {
-      throw new LuceneQueryException("Lucene Index is not available, currently indexing");
     }
 
     return entries;
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 c3eef4d..f60b667 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
@@ -52,6 +52,7 @@ import org.apache.geode.cache.lucene.LuceneQueryFactory;
 import org.apache.geode.cache.lucene.LuceneSerializer;
 import org.apache.geode.cache.lucene.internal.directory.DumpDirectoryFiles;
 import org.apache.geode.cache.lucene.internal.distributed.EntryScore;
+import org.apache.geode.cache.lucene.internal.distributed.IndexingInProgressFunction;
 import org.apache.geode.cache.lucene.internal.distributed.LuceneFunctionContext;
 import org.apache.geode.cache.lucene.internal.distributed.LuceneQueryFunction;
 import org.apache.geode.cache.lucene.internal.distributed.TopEntries;
@@ -124,6 +125,7 @@ public class LuceneServiceImpl implements InternalLuceneService {
     FunctionService.registerFunction(new LuceneQueryFunction());
     FunctionService.registerFunction(new LuceneGetPageFunction());
     FunctionService.registerFunction(new WaitUntilFlushedFunction());
+    FunctionService.registerFunction(new IndexingInProgressFunction());
     FunctionService.registerFunction(new DumpDirectoryFiles());
     registerDataSerializables();
   }
@@ -650,4 +652,50 @@ public class LuceneServiceImpl implements InternalLuceneService {
     }
     return true;
   }
+
+  public boolean isIndexingInProgress(String indexName, String regionPath) {
+    Region region = this.cache.getRegion(regionPath);
+    if (region == null) {
+      logger.info("Data region " + regionPath + " not found");
+      return false;
+    }
+    // If it is called from a client then we assume that all servers are already
+    // rolled to a version more than or equal to client's
+    // hence we don't need to validate the servers.
+    if (!cache.isClient()) {
+      // Also a check for PartitionedRegion. As we cannot use the same method calls to
+      // to get the members hosting the region for RR (future implementation)
+      if (region instanceof PartitionedRegion) {
+        PartitionedRegion dataRegion = (PartitionedRegion) region;
+        // Validate all members are Apache Geode v1.7.0 or above
+        Set<InternalDistributedMember> remoteMembers =
+            dataRegion.getRegionAdvisor().adviseAllPRNodes();
+        if (isAnyRemoteMemberVersionLessThanGeode1_7_0(remoteMembers)) {
+          throw new IllegalStateException(
+              LocalizedStrings.LuceneIndexingInProgress_CANNOT_BE_DETERMINED_BECAUSE_OF_VERSION_MISMATCH
+                  .toLocalizedString(regionPath));
+        }
+      }
+    }
+    Execution execution = FunctionService.onRegion(region);
+    ResultCollector resultCollector =
+        execution.setArguments(indexName).execute(IndexingInProgressFunction.ID);
+    List<Boolean> results = (List<Boolean>) resultCollector.getResult();
+    for (Boolean result : results) {
+      if (result == true) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean isAnyRemoteMemberVersionLessThanGeode1_7_0(
+      Set<InternalDistributedMember> remoteMembers) {
+    for (InternalDistributedMember remoteMember : remoteMembers) {
+      if (remoteMember.getVersionObject().ordinal() < Version.GEODE_170.ordinal()) {
+        return true;
+      }
+    }
+    return false;
+  }
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/IndexingInProgressFunction.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/IndexingInProgressFunction.java
new file mode 100644
index 0000000..934e775
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/IndexingInProgressFunction.java
@@ -0,0 +1,63 @@
+/*
+ * 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.distributed;
+
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.cache.lucene.LuceneIndex;
+import org.apache.geode.cache.lucene.LuceneService;
+import org.apache.geode.cache.lucene.LuceneServiceProvider;
+import org.apache.geode.internal.cache.execute.InternalFunction;
+
+public class IndexingInProgressFunction implements InternalFunction<Object> {
+
+  private static final long serialVersionUID = 1L;
+  public static final String ID = IndexingInProgressFunction.class.getName();
+
+  @Override
+  public void execute(FunctionContext<Object> context) {
+    RegionFunctionContext ctx = (RegionFunctionContext) context;
+    ResultSender<Boolean> resultSender = ctx.getResultSender();
+
+    Region region = ctx.getDataSet();
+    Cache cache = region.getCache();
+    String indexName = (String) ctx.getArguments();
+    if (indexName == null) {
+      throw new IllegalArgumentException("Missing index name");
+    }
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+    LuceneIndex luceneIndex = luceneService.getIndex(indexName, region.getFullPath());
+    if (luceneIndex == null) {
+      resultSender.lastResult(false);
+    } else {
+      resultSender.lastResult(luceneIndex.isIndexingInProgress());
+    }
+
+  }
+
+  @Override
+  public String getId() {
+    return ID;
+  }
+
+  @Override
+  public boolean optimizeForWrite() {
+    return true;
+  }
+}
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunction.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunction.java
index 716838d..ef8d4ce 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunction.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunction.java
@@ -49,6 +49,8 @@ import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PrimaryBucketException;
 import org.apache.geode.internal.cache.execute.InternalFunction;
 import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
+import org.apache.geode.internal.cache.execute.PartitionedRegionFunctionResultSender;
+import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 
 /**
@@ -65,10 +67,21 @@ public class LuceneQueryFunction implements InternalFunction<LuceneFunctionConte
 
   @Override
   public void execute(FunctionContext<LuceneFunctionContext> context) {
+    if (context.getResultSender() instanceof PartitionedRegionFunctionResultSender) {
+      PartitionedRegionFunctionResultSender resultSender =
+          (PartitionedRegionFunctionResultSender) context.getResultSender();
+      Version clientVersion = resultSender.getClientVersion();
+      if (clientVersion != null) { // is a client server connection
+        if (clientVersion.ordinal() < Version.GEODE_160.ordinal()) {
+          execute(context, true);
+          return;
+        }
+      }
+    }
     execute(context, false);
   }
 
-  private void handleException(LuceneIndexCreationInProgressException ex,
+  private void handleIfRetryNeededOnException(LuceneIndexCreationInProgressException ex,
       RegionFunctionContext ctx) {
     PartitionedRegion userDataRegion = (PartitionedRegion) ctx.getDataSet();
 
@@ -79,16 +92,18 @@ public class LuceneQueryFunction implements InternalFunction<LuceneFunctionConte
     // Hence the query waits for the repositories to be ready instead of throwing the exception
     if (!remoteMembers.isEmpty()) {
       for (InternalDistributedMember remoteMember : remoteMembers) {
-        if (remoteMember.getVersionObject().ordinal() <= Version.GEODE_160.ordinal()) {
+        if (remoteMember.getVersionObject().ordinal() < Version.GEODE_160.ordinal()) {
           // re-execute but wait till indexing is complete
           execute(ctx, true);
           return;
         }
       }
     }
-    // Will return the LuceneIndexCreationInProgressException as the new servers can handle this
-    // exception
-    throw ex;
+    // Cannot send IndexingInProgressException as the function may have been executed a new server
+    // on behalf of an old version client.
+    throw new FunctionException(new LuceneQueryException(
+        LocalizedStrings.LuceneQueryException_INDEX_NOT_AVAILABLE_CURRENTLY_INDEXING
+            .toLocalizedString()));
   }
 
   public void execute(FunctionContext<LuceneFunctionContext> context, boolean waitForRepository) {
@@ -156,10 +171,12 @@ public class LuceneQueryFunction implements InternalFunction<LuceneFunctionConte
       throw new InternalFunctionInvocationTargetException(e);
     } catch (LuceneIndexCreationInProgressException ex) {
       if (!waitForRepository) {
-        handleException(ex, ctx);
+        handleIfRetryNeededOnException(ex, ctx);
       } else {
-        logger.debug("The lucene query should have waited for the index to be created");
-        throw ex;
+        logger.warn("The lucene query should have waited for the index to be created");
+        throw new FunctionException(new LuceneQueryException(
+            LocalizedStrings.LuceneQueryException_INDEX_NOT_AVAILABLE_CURRENTLY_INDEXING
+                .toLocalizedString()));
       }
     }
   }
diff --git a/geode-lucene/src/main/resources/org/apache/geode/internal/sanctioned-geode-lucene-serializables.txt b/geode-lucene/src/main/resources/org/apache/geode/internal/sanctioned-geode-lucene-serializables.txt
index 2ef12c2..4f0a2b2 100755
--- a/geode-lucene/src/main/resources/org/apache/geode/internal/sanctioned-geode-lucene-serializables.txt
+++ b/geode-lucene/src/main/resources/org/apache/geode/internal/sanctioned-geode-lucene-serializables.txt
@@ -2,11 +2,12 @@ org/apache/geode/cache/lucene/LuceneIndexDestroyedException,false,indexName:java
 org/apache/geode/cache/lucene/LuceneIndexExistsException,false,indexName:java/lang/String,regionPath:java/lang/String
 org/apache/geode/cache/lucene/LuceneIndexNotFoundException,false,indexName:java/lang/String,regionPath:java/lang/String
 org/apache/geode/cache/lucene/LuceneQueryException,false
-org/apache/geode/cache/lucene/internal/cli/LuceneIndexStatus,false
+org/apache/geode/cache/lucene/internal/LuceneIndexCreationInProgressException,false
 org/apache/geode/cache/lucene/internal/cli/LuceneDestroyIndexInfo,false,definedDestroyOnly:boolean
 org/apache/geode/cache/lucene/internal/cli/LuceneFunctionSerializable,false,indexName:java/lang/String,regionPath:java/lang/String
 org/apache/geode/cache/lucene/internal/cli/LuceneIndexDetails,true,1,fieldAnalyzers:java/util/Map,indexStats:java/util/Map,searchableFieldNames:java/lang/String[],serializer:java/lang/String,serverName:java/lang/String,status:org/apache/geode/cache/lucene/internal/cli/LuceneIndexStatus
 org/apache/geode/cache/lucene/internal/cli/LuceneIndexInfo,true,1,fieldAnalyzers:java/lang/String[],searchableFieldNames:java/lang/String[],serializer:java/lang/String
+org/apache/geode/cache/lucene/internal/cli/LuceneIndexStatus,false
 org/apache/geode/cache/lucene/internal/cli/LuceneQueryInfo,true,1,defaultField:java/lang/String,keysOnly:boolean,limit:int,queryString:java/lang/String
 org/apache/geode/cache/lucene/internal/cli/LuceneSearchResults,false,exceptionFlag:boolean,exceptionMessage:java/lang/String,key:java/lang/String,score:float,value:java/lang/String
 org/apache/geode/cache/lucene/internal/cli/functions/LuceneCreateIndexFunction,true,3061443846664615818
@@ -15,7 +16,7 @@ org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunction,
 org/apache/geode/cache/lucene/internal/cli/functions/LuceneListIndexFunction,true,-2320432506763893879
 org/apache/geode/cache/lucene/internal/cli/functions/LuceneSearchIndexFunction,true,163818919780803222
 org/apache/geode/cache/lucene/internal/directory/DumpDirectoryFiles,true,1
+org/apache/geode/cache/lucene/internal/distributed/IndexingInProgressFunction,true,1
 org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunction,true,1
 org/apache/geode/cache/lucene/internal/distributed/WaitUntilFlushedFunction,true,1
 org/apache/geode/cache/lucene/internal/results/LuceneGetPageFunction,true,1
-org/apache/geode/cache/lucene/internal/LuceneIndexCreationInProgressException,false