You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by la...@apache.org on 2017/11/06 19:21:29 UTC

[geode] branch feature/GEODE-3930 updated (8fab192 -> 886da6a)

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

ladyvader pushed a change to branch feature/GEODE-3930
in repository https://gitbox.apache.org/repos/asf/geode.git.


 discard 8fab192  Cleanup IntelliJ refactoring glitch
 discard 135b958  Splotless
 discard 210e743  Added integration and DUnit tests for index creation on an existing region.
 discard ff8c066  Index creation after region creation works for a single member integration test
 discard bbcfdc8  Start of internal method to create a lucene index on an existing region
     new 886da6a  GEODE-3930: Create lucene index on an existing region

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (8fab192)
            \
             N -- N -- N   refs/heads/feature/GEODE-3930 (886da6a)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].

[geode] 01/01: GEODE-3930: Create lucene index on an existing region

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 886da6a1b298c7a7ae411899c8be7634c1dec763
Author: Dan Smith <up...@apache.org>
AuthorDate: Wed Nov 1 17:09:57 2017 -0700

    GEODE-3930: Create lucene index on an existing region
    
    - This is currently just an internal method to create the index on an existing
    region.  It does not actually index the existing data in the region.
    - Adding a parameter to LuceneIndexFactoryImpl.create to allow creation on
    an existing region.
---
 .../geode/internal/cache/AbstractRegion.java       |  8 +-
 .../geode/internal/cache/InternalRegion.java       |  3 +
 .../apache/geode/internal/cache/LocalRegion.java   | 16 ++--
 .../geode/internal/cache/PartitionedRegion.java    | 12 +--
 .../internal/cache/AbstractRegionJUnitTest.java    |  5 ++
 .../lucene/internal/LuceneIndexFactoryImpl.java    | 19 +++--
 .../cache/lucene/internal/LuceneIndexImpl.java     |  2 +
 .../lucene/internal/LuceneRegionListener.java      | 23 +-----
 .../cache/lucene/internal/LuceneServiceImpl.java   | 71 +++++++++++++++--
 .../lucene/internal/xml/LuceneIndexCreation.java   |  3 +-
 .../cache/lucene/LuceneIndexCreationDUnitTest.java | 93 ++++++++++++++++++++++
 .../lucene/LuceneIndexCreationIntegrationTest.java | 39 +++++++++
 .../internal/LuceneIndexFactoryImplJUnitTest.java  |  4 +-
 .../internal/LuceneServiceImplJUnitTest.java       |  5 +-
 14 files changed, 250 insertions(+), 53 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
index bc97a35..59fb160 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
@@ -869,8 +869,14 @@ public abstract class AbstractRegion implements InternalRegion, AttributesMutato
 
   @Override
   public void addAsyncEventQueueId(String asyncEventQueueId) {
+    addAsyncEventQueueId(asyncEventQueueId, false);
+  }
+
+  public void addAsyncEventQueueId(String asyncEventQueueId, boolean isInternal) {
     getAsyncEventQueueIds().add(asyncEventQueueId);
-    getVisibleAsyncEventQueueIds().add(asyncEventQueueId);
+    if (!isInternal) {
+      getVisibleAsyncEventQueueIds().add(asyncEventQueueId);
+    }
     setAllGatewaySenderIds();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
index 1b7c1b0..9dcd82b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
@@ -110,4 +110,7 @@ public interface InternalRegion<K, V>
 
   void invokeInvalidateCallbacks(final EnumListenerEvent eventType, final EntryEventImpl event,
       final boolean callDispatchListenerEvent);
+
+  void addCacheServiceProfile(CacheServiceProfile profile);
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 74bdb52..c2e916e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -215,6 +215,7 @@ import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
 import org.apache.geode.internal.offheap.annotations.Unretained;
 import org.apache.geode.internal.sequencelog.EntryLogger;
+import org.apache.geode.internal.util.concurrent.CopyOnWriteHashMap;
 import org.apache.geode.internal.util.concurrent.FutureResult;
 import org.apache.geode.internal.util.concurrent.StoppableCountDownLatch;
 import org.apache.geode.internal.util.concurrent.StoppableReadWriteLock;
@@ -541,7 +542,8 @@ public class LocalRegion extends AbstractRegion implements InternalRegion, Loade
     return this.stopper;
   }
 
-  Map<String, CacheServiceProfile> cacheServiceProfiles;
+  private final CopyOnWriteHashMap<String, CacheServiceProfile> cacheServiceProfiles =
+      new CopyOnWriteHashMap<>();
 
   private static String calcFullPath(String regionName, LocalRegion parentRegion) {
     StringBuilder buf = null;
@@ -634,9 +636,9 @@ public class LocalRegion extends AbstractRegion implements InternalRegion, Loade
     this.isUsedForParallelGatewaySenderQueue =
         internalRegionArgs.isUsedForParallelGatewaySenderQueue();
     this.serialGatewaySender = internalRegionArgs.getSerialGatewaySender();
-    this.cacheServiceProfiles =
-        internalRegionArgs.getCacheServiceProfiles() == null ? Collections.emptyMap()
-            : Collections.unmodifiableMap(internalRegionArgs.getCacheServiceProfiles());
+    if (internalRegionArgs.getCacheServiceProfiles() != null) {
+      this.cacheServiceProfiles.putAll(internalRegionArgs.getCacheServiceProfiles());
+    }
 
     if (!isUsedForMetaRegion && !isUsedForPartitionedRegionAdmin
         && !isUsedForPartitionedRegionBucket && !isUsedForSerialGatewaySenderQueue
@@ -10399,7 +10401,11 @@ public class LocalRegion extends AbstractRegion implements InternalRegion, Loade
   }
 
   Map<String, CacheServiceProfile> getCacheServiceProfiles() {
-    return Collections.unmodifiableMap(this.cacheServiceProfiles);
+    return this.cacheServiceProfiles.getSnapshot();
+  }
+
+  public void addCacheServiceProfile(CacheServiceProfile profile) {
+    this.cacheServiceProfiles.put(profile.getId(), profile);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
index 7720bfa..da8697b 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
@@ -1218,7 +1218,11 @@ public class PartitionedRegion extends LocalRegion
   }
 
   public void addAsyncEventQueueId(String asyncEventQueueId) {
-    super.addAsyncEventQueueId(asyncEventQueueId);
+    addAsyncEventQueueId(asyncEventQueueId, false);
+  }
+
+  public void addAsyncEventQueueId(String asyncEventQueueId, boolean isInternal) {
+    super.addAsyncEventQueueId(asyncEventQueueId, isInternal);
     new UpdateAttributesProcessor(this).distribute();
     ((PartitionedRegion) this).distributeUpdatedProfileOnSenderCreation();
     GatewaySender sender = getCache()
@@ -4966,10 +4970,8 @@ public class PartitionedRegion extends LocalRegion
 
     fillInProfile((PartitionProfile) profile);
 
-    if (cacheServiceProfiles != null) {
-      for (CacheServiceProfile csp : cacheServiceProfiles.values()) {
-        profile.addCacheServiceProfile(csp);
-      }
+    for (CacheServiceProfile csp : getCacheServiceProfiles().values()) {
+      profile.addCacheServiceProfile(csp);
     }
 
     profile.isOffHeap = getOffHeap();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
index 17acc95..a0c6e32 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
@@ -657,5 +657,10 @@ public class AbstractRegionJUnitTest {
     public boolean supportsConcurrencyChecks() {
       throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void addCacheServiceProfile(CacheServiceProfile profile) {
+      throw new UnsupportedOperationException();
+    }
   }
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
index 429da38..4155f13 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
@@ -23,7 +23,7 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
-public class LuceneIndexFactoryImpl implements org.apache.geode.cache.lucene.LuceneIndexFactory {
+public class LuceneIndexFactoryImpl implements LuceneIndexFactory {
   private final LuceneServiceImpl service;
   private final Map<String, Analyzer> fields = new LinkedHashMap<String, Analyzer>();
   private LuceneSerializer serializer;
@@ -34,12 +34,12 @@ public class LuceneIndexFactoryImpl implements org.apache.geode.cache.lucene.Luc
   }
 
   @Override
-  public LuceneIndexFactory addField(final String name) {
+  public LuceneIndexFactoryImpl addField(final String name) {
     return addField(name, new StandardAnalyzer());
   }
 
   @Override
-  public LuceneIndexFactory setFields(final String... fields) {
+  public LuceneIndexFactoryImpl setFields(final String... fields) {
     this.fields.clear();
     for (String field : fields) {
       addField(field);
@@ -48,13 +48,13 @@ public class LuceneIndexFactoryImpl implements org.apache.geode.cache.lucene.Luc
   }
 
   @Override
-  public LuceneIndexFactory addField(final String name, final Analyzer analyzer) {
+  public LuceneIndexFactoryImpl addField(final String name, final Analyzer analyzer) {
     fields.put(name, analyzer);
     return this;
   }
 
   @Override
-  public LuceneIndexFactory setFields(final Map<String, Analyzer> fieldMap) {
+  public LuceneIndexFactoryImpl setFields(final Map<String, Analyzer> fieldMap) {
     this.fields.clear();
     this.fields.putAll(fieldMap);
     return this;
@@ -62,11 +62,16 @@ public class LuceneIndexFactoryImpl implements org.apache.geode.cache.lucene.Luc
 
   @Override
   public void create(final String indexName, final String regionPath) {
-    service.createIndex(indexName, regionPath, fields, serializer);
+    this.create(indexName, regionPath, false);
+  }
+
+  public void create(final String indexName, final String regionPath,
+      boolean allowOnExistingRegion) {
+    service.createIndex(indexName, regionPath, fields, serializer, allowOnExistingRegion);
   }
 
   @Override
-  public LuceneIndexFactory setLuceneSerializer(LuceneSerializer luceneSerializer) {
+  public LuceneIndexFactoryImpl setLuceneSerializer(LuceneSerializer luceneSerializer) {
     this.serializer = luceneSerializer;
     return this;
   }
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 2110ca5..fe4ca19 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
@@ -24,6 +24,8 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.geode.InternalGemFireError;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.EvictionAlgorithm;
+import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.asyncqueue.AsyncEventQueue;
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
index df9dca0..bec0dd2 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRegionListener.java
@@ -20,8 +20,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.lucene.analysis.Analyzer;
 
 import org.apache.geode.cache.AttributesFactory;
-import org.apache.geode.cache.EvictionAlgorithm;
-import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl;
@@ -83,21 +81,7 @@ public class LuceneRegionListener implements RegionListener {
 
     if (path.equals(this.regionPath) && this.beforeCreateInvoked.compareAndSet(false, true)) {
 
-      if (!attrs.getDataPolicy().withPartitioning()) {
-        // replicated region
-        throw new UnsupportedOperationException(
-            "Lucene indexes on replicated regions are not supported");
-      }
-
-      // For now we cannot support eviction with local destroy.
-      // Eviction with overflow to disk still needs to be supported
-      EvictionAttributes evictionAttributes = attrs.getEvictionAttributes();
-      EvictionAlgorithm evictionAlgorithm = evictionAttributes.getAlgorithm();
-      if (evictionAlgorithm != EvictionAlgorithm.NONE
-          && evictionAttributes.getAction().isLocalDestroy()) {
-        throw new UnsupportedOperationException(
-            "Lucene indexes on regions with eviction and action local destroy are not supported");
-      }
+      LuceneServiceImpl.validateRegionAttributes(attrs);
 
       String aeqId = LuceneServiceImpl.getUniqueIndexName(this.indexName, this.regionPath);
       if (!attrs.getAsyncEventQueueIds().contains(aeqId)) {
@@ -124,11 +108,6 @@ public class LuceneRegionListener implements RegionListener {
     if (region.getFullPath().equals(this.regionPath)
         && this.afterCreateInvoked.compareAndSet(false, true)) {
       this.service.afterDataRegionCreated(this.luceneIndex);
-      String aeqId = LuceneServiceImpl.getUniqueIndexName(this.indexName, this.regionPath);
-      AsyncEventQueueImpl aeq = (AsyncEventQueueImpl) cache.getAsyncEventQueue(aeqId);
-      AbstractPartitionedRepositoryManager repositoryManager =
-          (AbstractPartitionedRepositoryManager) luceneIndex.getRepositoryManager();
-      repositoryManager.allowRepositoryComputation();
     }
   }
 
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 985e3ad..22f498c 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
@@ -32,7 +32,10 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
+import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.EvictionAlgorithm;
+import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl;
@@ -63,6 +66,8 @@ import org.apache.geode.internal.DSFIDFactory;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.cache.CacheService;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.RegionListener;
 import org.apache.geode.internal.cache.extension.Extensible;
 import org.apache.geode.internal.cache.xmlcache.XmlGenerator;
@@ -202,20 +207,21 @@ public class LuceneServiceImpl implements InternalLuceneService {
   }
 
   public void createIndex(String indexName, String regionPath, Map<String, Analyzer> fieldAnalyzers,
-      LuceneSerializer serializer) {
+      LuceneSerializer serializer, boolean allowOnExistingRegion) {
     if (fieldAnalyzers == null || fieldAnalyzers.isEmpty()) {
       throw new IllegalArgumentException("At least one field must be indexed");
     }
     Analyzer analyzer = new PerFieldAnalyzerWrapper(new StandardAnalyzer(), fieldAnalyzers);
     Set<String> fieldsSet = fieldAnalyzers.keySet();
-    String[] fields = (String[]) fieldsSet.toArray(new String[fieldsSet.size()]);
+    String[] fields = fieldsSet.toArray(new String[fieldsSet.size()]);
 
-    createIndex(indexName, regionPath, analyzer, fieldAnalyzers, serializer, fields);
+    createIndex(indexName, regionPath, analyzer, fieldAnalyzers, serializer, allowOnExistingRegion,
+        fields);
   }
 
   public void createIndex(final String indexName, String regionPath, final Analyzer analyzer,
       final Map<String, Analyzer> fieldAnalyzers, final LuceneSerializer serializer,
-      final String... fields) {
+      boolean allowOnExistingRegion, final String... fields) {
 
     if (!regionPath.startsWith("/")) {
       regionPath = "/" + regionPath;
@@ -224,14 +230,59 @@ public class LuceneServiceImpl implements InternalLuceneService {
     registerDefinedIndex(indexName, regionPath, new LuceneIndexCreationProfile(indexName,
         regionPath, fields, analyzer, fieldAnalyzers, serializer));
 
-    Region region = cache.getRegion(regionPath);
-    if (region != null) {
+    PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionPath);
+
+    LuceneRegionListener regionListener = new LuceneRegionListener(this, cache, indexName,
+        regionPath, fields, analyzer, fieldAnalyzers, serializer);
+    if (region == null) {
+      cache.addRegionListener(regionListener);
+      return;
+    }
+
+    if (!allowOnExistingRegion) {
       definedIndexMap.remove(LuceneServiceImpl.getUniqueIndexName(indexName, regionPath));
       throw new IllegalStateException("The lucene index must be created before region");
     }
 
-    cache.addRegionListener(new LuceneRegionListener(this, cache, indexName, regionPath, fields,
+
+    createIndexOnExistingRegion(region, indexName, regionPath, fields, analyzer, fieldAnalyzers,
+        serializer);
+
+  }
+
+  private void createIndexOnExistingRegion(PartitionedRegion region, String indexName,
+      String regionPath, String[] fields, Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers,
+      LuceneSerializer serializer) {
+    validateRegionAttributes(region.getAttributes());
+
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
+    region.addAsyncEventQueueId(aeqId, true);
+
+    region.addCacheServiceProfile(new LuceneIndexCreationProfile(indexName, regionPath, fields,
         analyzer, fieldAnalyzers, serializer));
+
+    LuceneIndexImpl luceneIndex = beforeDataRegionCreated(indexName, regionPath,
+        region.getAttributes(), analyzer, fieldAnalyzers, aeqId, serializer, fields);
+
+    afterDataRegionCreated(luceneIndex);
+  }
+
+  static void validateRegionAttributes(RegionAttributes attrs) {
+    if (!attrs.getDataPolicy().withPartitioning()) {
+      // replicated region
+      throw new UnsupportedOperationException(
+          "Lucene indexes on replicated regions are not supported");
+    }
+
+    // For now we cannot support eviction with local destroy.
+    // Eviction with overflow to disk still needs to be supported
+    EvictionAttributes evictionAttributes = attrs.getEvictionAttributes();
+    EvictionAlgorithm evictionAlgorithm = evictionAttributes.getAlgorithm();
+    if (evictionAlgorithm != EvictionAlgorithm.NONE
+        && evictionAttributes.getAction().isLocalDestroy()) {
+      throw new UnsupportedOperationException(
+          "Lucene indexes on regions with eviction and action local destroy are not supported");
+    }
   }
 
   /**
@@ -246,6 +297,12 @@ public class LuceneServiceImpl implements InternalLuceneService {
       this.managementListener.afterIndexCreated(index);
     }
 
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(index.getName(), index.getRegionPath());
+    AsyncEventQueueImpl aeq = (AsyncEventQueueImpl) cache.getAsyncEventQueue(aeqId);
+    AbstractPartitionedRepositoryManager repositoryManager =
+        (AbstractPartitionedRepositoryManager) index.getRepositoryManager();
+    repositoryManager.allowRepositoryComputation();
+
   }
 
   public LuceneIndexImpl beforeDataRegionCreated(final String indexName, final String regionPath,
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java
index 7af43fb..3ed1bba 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -17,7 +17,6 @@ package org.apache.geode.cache.lucene.internal.xml;
 
 import java.util.*;
 
-import org.apache.geode.cache.Declarable;
 import org.apache.geode.cache.lucene.LuceneIndexExistsException;
 import org.apache.geode.cache.lucene.LuceneSerializer;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -100,7 +99,7 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
         : new PerFieldAnalyzerWrapper(new StandardAnalyzer(), this.fieldAnalyzers);
     try {
       service.createIndex(getName(), getRegionPath(), analyzer, this.fieldAnalyzers,
-          getLuceneSerializer(), getFieldNames());
+          getLuceneSerializer(), false, getFieldNames());
     } catch (LuceneIndexExistsException e) {
       logger
           .info(LocalizedStrings.LuceneIndexCreation_IGNORING_DUPLICATE_INDEX_CREATION_0_ON_REGION_1
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
index 948ba14..fd296fd 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
@@ -14,8 +14,12 @@
  */
 package org.apache.geode.cache.lucene;
 
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.lucene.internal.LuceneIndexFactoryImpl;
 import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
+import org.apache.geode.cache.lucene.test.TestObject;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.test.dunit.SerializableRunnableIF;
 import org.apache.geode.test.junit.categories.DistributedTest;
@@ -25,15 +29,20 @@ import junitparams.Parameters;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.hamcrest.Matchers;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.*;
 import static junitparams.JUnitParamsRunner.$;
@@ -43,6 +52,9 @@ import static org.junit.Assert.*;
 @RunWith(JUnitParamsRunner.class)
 public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
 
+  @Rule
+  public transient ExpectedException expectedException = ExpectedException.none();
+
   private Object[] parametersForMultipleIndexCreates() {
     Integer[] numIndexes = {1, 2, 10};
     RegionTestableType[] regionTestTypes = getListOfRegionTestTypes();
@@ -248,6 +260,67 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
     dataStore2.invoke(() -> initDataStore(createIndex, regionType));
   }
 
+  @Test
+  @Parameters({"PARTITION", "PARTITION_REDUNDANT"})
+  public void creatingIndexAfterRegionAndStartingUpSecondMemberSucceeds(
+      RegionTestableType regionType) {
+    dataStore1.invoke(() -> {
+      regionType.createDataStore(getCache(), REGION_NAME);
+      createIndexAfterRegion("field1");
+    });
+
+    dataStore2.invoke(() -> {
+      createIndexAfterRegion("field1");
+      regionType.createDataStore(getCache(), REGION_NAME);
+    });
+    dataStore1.invoke(() -> {
+      putEntryAndQuery();
+    });
+  }
+
+  @Test()
+  @Parameters({"PARTITION", "PARTITION_REDUNDANT"})
+  public void creatingIndexAfterRegionAndStartingUpSecondMemberWithoutIndexFails(
+      RegionTestableType regionType) {
+    dataStore1.invoke(() -> {
+      regionType.createDataStore(getCache(), REGION_NAME);
+      createIndexAfterRegion("field1");
+    });
+
+    expectedException.expectCause(Matchers.instanceOf(IllegalStateException.class));
+    dataStore2.invoke(() -> {
+      regionType.createDataStore(getCache(), REGION_NAME);
+      createIndexAfterRegion("field1");
+    });
+
+    dataStore1.invoke(() -> {
+      putEntryAndQuery();
+    });
+  }
+
+  @Test
+  @Parameters({"PARTITION", "PARTITION_REDUNDANT"})
+  public void creatingIndexAfterRegionInTwoMembersSucceed(RegionTestableType regionType) {
+    dataStore1.invoke(() -> {
+      regionType.createDataStore(getCache(), REGION_NAME);
+    });
+
+    dataStore2.invoke(() -> {
+      regionType.createDataStore(getCache(), REGION_NAME);
+    });
+
+    dataStore1.invoke(() -> {
+      createIndexAfterRegion("field1");
+    });
+
+    dataStore2.invoke(() -> {
+      createIndexAfterRegion("field1");
+    });
+
+    dataStore1.invoke(() -> {
+      putEntryAndQuery();
+    });
+  }
 
   @Test
   @Parameters(method = "getXmlAndExceptionMessages")
@@ -544,4 +617,24 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
   protected void verifyAsyncEventQueues(final int expectedSize) {
     assertEquals(expectedSize, getCache().getAsyncEventQueues(false).size());
   }
+
+  private void createIndexAfterRegion(String... fields) {
+    LuceneService luceneService = LuceneServiceProvider.get(getCache());
+    LuceneIndexFactoryImpl indexFactory =
+        (LuceneIndexFactoryImpl) luceneService.createIndexFactory();
+    indexFactory.setFields(fields).create(INDEX_NAME, REGION_NAME, true);
+  }
+
+  private void putEntryAndQuery() throws InterruptedException, LuceneQueryException {
+    Cache cache = getCache();
+    Region region = cache.getRegion(REGION_NAME);
+    region.put("key1", new TestObject("field1Value", "field2Value"));
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+    luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, 1, TimeUnit.MINUTES);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory().create(INDEX_NAME,
+        REGION_NAME, "field1:field1Value", "field1");
+    assertEquals(Collections.singletonList("key1"), query.findKeys());
+  }
+
+
 }
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
index 1d688fb..5f4e74b 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
@@ -39,6 +40,7 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.internal.LuceneIndexCreationProfile;
+import org.apache.geode.cache.lucene.internal.LuceneIndexFactoryImpl;
 import org.apache.geode.cache.lucene.internal.LuceneIndexImplFactory;
 import org.apache.geode.cache.lucene.internal.LuceneRawIndex;
 import org.apache.geode.cache.lucene.internal.LuceneRawIndexFactory;
@@ -184,6 +186,43 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     createIndex("field1", "field2", "field3");
   }
 
+  @Test()
+  public void canCreateLuceneIndexAfterRegionCreatedIfAllowFlagIsSet()
+      throws IOException, ParseException, InterruptedException, LuceneQueryException {
+    Region region = createRegion();
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+    final LuceneIndexFactoryImpl indexFactory =
+        (LuceneIndexFactoryImpl) luceneService.createIndexFactory();
+    indexFactory.setFields("field1", "field2").create(INDEX_NAME, REGION_NAME, true);
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    assertNotNull(index);
+
+    region.put("key1", new TestObject("hello", "world"));
+    luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, 1, TimeUnit.MINUTES);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory().create(INDEX_NAME,
+        REGION_NAME, "field1:hello", "field1");
+
+    assertEquals(Collections.singletonList("key1"), query.findKeys());
+  }
+
+
+  @Test()
+  public void creatingDuplicateLuceneIndexFails()
+      throws IOException, ParseException, InterruptedException, LuceneQueryException {
+    Region region = createRegion();
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+    final LuceneIndexFactoryImpl indexFactory =
+        (LuceneIndexFactoryImpl) luceneService.createIndexFactory();
+    indexFactory.setFields("field1", "field2").create(INDEX_NAME, REGION_NAME, true);
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    assertNotNull(index);
+
+    expectedException.expect(LuceneIndexExistsException.class);
+    indexFactory.setFields("field1", "field2").create(INDEX_NAME, REGION_NAME, true);
+  }
+
   @Test
   public void cannotCreateLuceneIndexForReplicateRegion() throws IOException, ParseException {
     try {
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImplJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImplJUnitTest.java
index 67759db..bafb60a 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImplJUnitTest.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.cache.lucene.internal;
 
-import static org.junit.Assert.*;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
@@ -37,7 +36,8 @@ public class LuceneIndexFactoryImplJUnitTest {
     LuceneIndexFactory factory = new LuceneIndexFactoryImpl(service);
     factory.setLuceneSerializer(serializer);
     factory.create("index", "region");
-    Mockito.verify(service).createIndex(eq("index"), eq("region"), any(), eq(serializer));
+    Mockito.verify(service).createIndex(eq("index"), eq("region"), any(), eq(serializer),
+        eq(false));
   }
 
 }
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java
index c5ca8f8..736b3ca 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneServiceImplJUnitTest.java
@@ -62,7 +62,8 @@ public class LuceneServiceImplJUnitTest {
     factory.setLuceneSerializer(serializer);
     factory.setFields("field1", "field2");
     factory.create("index", "region");
-    Mockito.verify(service).createIndex(eq("index"), eq("region"), any(), eq(serializer));
+    Mockito.verify(service).createIndex(eq("index"), eq("region"), any(), eq(serializer),
+        eq(false));
   }
 
   @Test
@@ -74,7 +75,7 @@ public class LuceneServiceImplJUnitTest {
   @Test
   public void shouldThrowIllegalArgumentExceptionIfFieldsMapIsMissing() {
     thrown.expect(IllegalArgumentException.class);
-    service.createIndex("index", "region", Collections.emptyMap(), null);
+    service.createIndex("index", "region", Collections.emptyMap(), null, false);
   }
 
   @Test

-- 
To stop receiving notification emails like this one, please contact
"commits@geode.apache.org" <co...@geode.apache.org>.