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

[geode] branch feature/GEODE-3930 updated: Index creation after region creation works for a single member integration test

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

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


The following commit(s) were added to refs/heads/feature/GEODE-3930 by this push:
     new dc5d4e9  Index creation after region creation works for a single member integration test
dc5d4e9 is described below

commit dc5d4e9127bd0ddad27bc275af1850d974bebfea
Author: Dan Smith <up...@apache.org>
AuthorDate: Thu Nov 2 12:37:41 2017 -0700

    Index creation after region creation works for a single member integration test
---
 .../apache/geode/internal/cache/AbstractRegion.java  |  8 +++++++-
 .../apache/geode/internal/cache/InternalRegion.java  |  2 ++
 .../org/apache/geode/internal/cache/LocalRegion.java | 15 ++++++++++-----
 .../geode/internal/cache/PartitionedRegion.java      | 13 ++++++++-----
 .../lucene/internal/LuceneIndexFactoryImpl.java      |  2 +-
 .../cache/lucene/internal/LuceneRegionListener.java  |  5 -----
 .../cache/lucene/internal/LuceneServiceImpl.java     | 20 +++++++++++++-------
 .../lucene/LuceneIndexCreationIntegrationTest.java   | 20 +++++++++++++++++---
 8 files changed, 58 insertions(+), 27 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 e6fdaaf..f3971d3 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 Region, RegionAttributes, Attrib
 
   @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 4af148c..b713135 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
@@ -41,4 +41,6 @@ public interface InternalRegion<K, V> extends Region<K, V>, HasCachePerfStats {
 
   RegionEntry getRegionEntry(K key);
 
+  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 7df7da2..7e703d0 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;
@@ -540,7 +541,7 @@ 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;
@@ -633,9 +634,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
@@ -10376,7 +10377,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 dcb31b4..e3e399f 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,12 @@ 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 +4971,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-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 6353ce7..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
@@ -67,7 +67,7 @@ public class LuceneIndexFactoryImpl implements LuceneIndexFactory {
 
   public void create(final String indexName, final String regionPath,
       boolean allowOnExistingRegion) {
-    service.createIndex(indexName, regionPath, fields, serializer, false);
+    service.createIndex(indexName, regionPath, fields, serializer, allowOnExistingRegion);
   }
 
   @Override
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 fa44f63..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
@@ -108,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 e0ab851..4d2bf31 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
@@ -66,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;
@@ -228,7 +230,7 @@ public class LuceneServiceImpl implements InternalLuceneService {
     registerDefinedIndex(indexName, regionPath, new LuceneIndexCreationProfile(indexName,
         regionPath, fields, analyzer, fieldAnalyzers, serializer));
 
-    Region region = cache.getRegion(regionPath);
+    PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionPath);
 
     LuceneRegionListener regionListener = new LuceneRegionListener(this, cache, indexName,
         regionPath, fields, analyzer, fieldAnalyzers, serializer);
@@ -248,23 +250,21 @@ public class LuceneServiceImpl implements InternalLuceneService {
 
   }
 
-  private void createIndexOnExistingRegion(Region region, String indexName, String regionPath,
+  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.getAttributesMutator().addAsyncEventQueueId(aeqId);
+    region.addAsyncEventQueueId(aeqId, true);
 
-    // Add index creation profile
-    internalRegionArgs.addCacheServiceProfile(new LuceneIndexCreationProfile(indexName, regionPath,
+    region.addCacheServiceProfile(new LuceneIndexCreationProfile(indexName, regionPath,
         fields, analyzer, fieldAnalyzers, serializer));
 
     LuceneIndexImpl luceneIndex = beforeDataRegionCreated(indexName, regionPath,
         region.getAttributes(), analyzer, fieldAnalyzers, aeqId, serializer, fields);
 
-    // Add internal async event id
-    internalRegionArgs.addInternalAsyncEventQueueId(aeqId);
+    afterDataRegionCreated(luceneIndex);
   }
 
   static void validateRegionAttributes(RegionAttributes attrs) {
@@ -297,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/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
index d0a1da8..8a57666 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;
 
@@ -187,11 +188,24 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
 
   @Test()
   public void canCreateLuceneIndexAfterRegionCreatedIfAllowFlagIsSet()
-      throws IOException, ParseException {
-    createRegion();
+      throws IOException, ParseException, InterruptedException, LuceneQueryException {
+    Region region = createRegion();
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
     final LuceneIndexFactoryImpl indexFactory =
-        (LuceneIndexFactoryImpl) LuceneServiceProvider.get(cache).createIndexFactory();
+        (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

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