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

[geode] 03/03: The crazy continues.... Why two different ExtendedLuceneIndexForPartitionedRegion?

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

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

commit 13ce16044d7366d6b83ff0389bcae0310416be76
Author: Udo <uk...@pivotal.io>
AuthorDate: Fri Mar 16 09:42:44 2018 -0700

    The crazy continues.... Why two different ExtendedLuceneIndexForPartitionedRegion?
---
 .../lucene/internal/IndexRepositoryFactory.java    |  1 +
 .../internal/LuceneIndexForPartitionedRegion.java  | 35 +++++++++++++++-------
 .../cache/lucene/internal/LuceneIndexImpl.java     |  3 +-
 .../cache/lucene/internal/LuceneRawIndex.java      | 26 +++++++++++++++-
 .../lucene/internal/LuceneRawIndexFactory.java     | 15 ++++++++++
 .../lucene/internal/LuceneRegionListener.java      |  1 +
 .../cache/lucene/internal/LuceneServiceImpl.java   | 21 +++++++++----
 .../internal/PartitionedRepositoryManager.java     |  1 +
 .../lucene/internal/LuceneIndexFactorySpy.java     | 27 ++++++++++++-----
 .../internal/LuceneServiceImplJUnitTest.java       |  9 ++++++
 10 files changed, 114 insertions(+), 25 deletions(-)

diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java
index fcb1620..b3aeeee 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java
@@ -50,6 +50,7 @@ public class IndexRepositoryFactory {
   public IndexRepository computeIndexRepository(final Integer bucketId, LuceneSerializer serializer,
       InternalLuceneIndex index, PartitionedRegion userRegion, final IndexRepository oldRepository)
       throws IOException {
+    System.out.println(this+".computeIndexRepository bucketId: "+bucketId);
     return ((LuceneIndexForPartitionedRegion) index).computeIndexRepository(bucketId, serializer,
         userRegion, oldRepository);
   }
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 0f5fda4..b338595 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
@@ -85,9 +85,10 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   public LuceneIndexForPartitionedRegion(String indexName, String regionPath, InternalCache cache,
-      Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers, LuceneSerializer serializer,
-      RegionAttributes attributes, String aeqId, String[] fields,
-      ExecutorService waitingThreadPool) {
+                                         Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers,
+                                         LuceneSerializer serializer,
+                                         RegionAttributes attributes, String aeqId, String[] fields,
+                                         ExecutorService waitingThreadPool) {
     this(indexName, regionPath, cache);
     this.waitingThreadPool = waitingThreadPool;
     this.setSearchableFields(fields);
@@ -95,6 +96,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     this.setFieldAnalyzers(fieldAnalyzers);
     this.setLuceneSerializer(serializer);
     this.setupRepositoryManager(serializer);
+    System.out.println(this+".LuceneIndexForPartitionedRegion construct  " + this.repositoryManager);
     this.createAEQ(attributes, aeqId);
   }
 
@@ -139,6 +141,11 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     if (!fileRegionExists(fileRegionName)) {
       fileAndChunkRegion = createRegion(fileRegionName, regionShortCut, this.regionPath,
           partitionAttributes, regionAttributes, lucenePrimaryBucketListener);
+      System.err
+          .println(this + ".createLuceneListenersAndFileChunkRegions   fileAndChunkRegion: "
+              + fileAndChunkRegion);
+    } else {
+      fileAndChunkRegion = this.cache.getRegion(fileRegionName);
     }
 
     fileSystemStats
@@ -182,9 +189,11 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   protected <K, V> Region<K, V> createRegion(final String regionName,
-      final RegionShortcut regionShortCut, final String colocatedWithRegionName,
-      final PartitionAttributes partitionAttributes, final RegionAttributes regionAttributes,
-      PartitionListener lucenePrimaryBucketListener) {
+                                             final RegionShortcut regionShortCut,
+                                             final String colocatedWithRegionName,
+                                             final PartitionAttributes partitionAttributes,
+                                             final RegionAttributes regionAttributes,
+                                             PartitionListener lucenePrimaryBucketListener) {
     PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory();
     if (lucenePrimaryBucketListener != null) {
       partitionAttributesFactory.addPartitionListener(lucenePrimaryBucketListener);
@@ -204,12 +213,13 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return createRegion(regionName, attributes);
   }
 
-  public void close() {}
+  public void close() {
+  }
 
   @Override
   public void dumpFiles(final String directory) {
     ResultCollector results = FunctionService.onRegion(getDataRegion())
-        .setArguments(new String[] {directory, indexName}).execute(DumpDirectoryFiles.ID);
+        .setArguments(new String[]{directory, indexName}).execute(DumpDirectoryFiles.ID);
     results.getResult();
   }
 
@@ -283,7 +293,9 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   public IndexRepository computeIndexRepository(final Integer bucketId, LuceneSerializer serializer,
-      PartitionedRegion userRegion, final IndexRepository oldRepository) throws IOException {
+                                                PartitionedRegion userRegion,
+                                                final IndexRepository oldRepository)
+      throws IOException {
     final PartitionedRegion fileRegion = getFileAndChunkRegion();
     BucketRegion fileAndChunkBucket = getFileBucketRegion(bucketId, fileRegion);
 
@@ -351,6 +363,8 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
 
   private BucketRegion getFileBucketRegion(Integer bucketId, PartitionedRegion fileRegion) {
     // We need to ensure that all members have created the fileAndChunk region before continuing
+    System.err.println(
+        this + ".getFileBucketRegion for bucket: " + bucketId + "  fileRegion:" + fileRegion);
     Region prRoot = PartitionedRegionHelper.getPRRoot(fileRegion.getCache());
     PartitionRegionConfig prConfig =
         (PartitionRegionConfig) prRoot.get(fileRegion.getRegionIdentifier());
@@ -362,7 +376,8 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   private boolean reindexUserDataRegion(Integer bucketId, PartitionedRegion userRegion,
-      PartitionedRegion fileRegion, BucketRegion dataBucket, IndexRepository repo)
+                                        PartitionedRegion fileRegion, BucketRegion dataBucket,
+                                        IndexRepository repo)
       throws IOException {
     Set<IndexRepository> affectedRepos = new HashSet<IndexRepository>();
 
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
index e58c21f..45ecc78 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
@@ -143,6 +143,7 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
   public void initialize() {
     /* create index region */
     dataRegion = assignDataRegion();
+    System.err.println(this+".initialize  DataRegion: "+dataRegion+"  repositoryManager: "+repositoryManager);
     createLuceneListenersAndFileChunkRegions((PartitionedRepositoryManager) repositoryManager);
     addExtension(dataRegion);
   }
@@ -170,8 +171,8 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     }
     return createAEQ(createAEQFactory(attributes), aeqId);
   }
-
   private AsyncEventQueue createAEQ(AsyncEventQueueFactoryImpl factory, String aeqId) {
+    System.err.println(this+".createAEQ RepositoryManager: "+repositoryManager);
     LuceneEventListener listener = new LuceneEventListener(cache, repositoryManager);
     factory.setGatewayEventSubstitutionListener(new LuceneEventSubstitutionFilter());
     AsyncEventQueue indexQueue = factory.create(aeqId, listener);
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 91011f3..54f16e9 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
@@ -14,6 +14,12 @@
  */
 package org.apache.geode.cache.lucene.internal;
 
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.lucene.analysis.Analyzer;
+
+import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.lucene.LuceneSerializer;
 import org.apache.geode.cache.lucene.internal.repository.RepositoryManager;
 import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
@@ -22,10 +28,27 @@ import org.apache.geode.internal.cache.PartitionedRegion;
 
 public class LuceneRawIndex extends LuceneIndexImpl {
 
+  private ExecutorService waitingThreadPool;
+
   protected LuceneRawIndex(String indexName, String regionPath, InternalCache cache) {
     super(indexName, regionPath, cache);
   }
 
+  public LuceneRawIndex(String indexName, String regionPath, InternalCache cache, Analyzer analyzer,
+                        Map<String, Analyzer> fieldAnalyzers,
+                        LuceneSerializer serializer, RegionAttributes attributes, String aeqId,
+                        String[] fields,
+                        ExecutorService waitingThreadPool) {
+    super(indexName, regionPath, cache);
+    this.waitingThreadPool = waitingThreadPool;
+    this.setSearchableFields(fields);
+    this.setAnalyzer(analyzer);
+    this.setFieldAnalyzers(fieldAnalyzers);
+    this.setLuceneSerializer(serializer);
+    this.setupRepositoryManager(serializer);
+    this.createAEQ(attributes, aeqId);
+  }
+
   @Override
   protected RepositoryManager createRepositoryManager(LuceneSerializer luceneSerializer) {
     HeterogeneousLuceneSerializer mapper = (HeterogeneousLuceneSerializer) luceneSerializer;
@@ -49,7 +72,8 @@ public class LuceneRawIndex extends LuceneIndexImpl {
   }
 
   @Override
-  public void destroy(boolean initiator) {}
+  public void destroy(boolean initiator) {
+  }
 
   @Override
   public boolean isIndexAvailable(int id) {
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java
index 4a92049..e319fb9 100755
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneRawIndexFactory.java
@@ -14,6 +14,13 @@
  */
 package org.apache.geode.cache.lucene.internal;
 
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.lucene.analysis.Analyzer;
+
+import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.lucene.LuceneSerializer;
 import org.apache.geode.internal.cache.InternalCache;
 
 public class LuceneRawIndexFactory extends LuceneIndexImplFactory {
@@ -21,4 +28,12 @@ public class LuceneRawIndexFactory extends LuceneIndexImplFactory {
   public LuceneIndexImpl create(String indexName, String regionPath, InternalCache cache) {
     return new LuceneRawIndex(indexName, regionPath, cache);
   }
+
+  public LuceneIndexImpl create(String indexName, String regionPath, InternalCache cache,
+                                Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers, LuceneSerializer serializer,
+                                RegionAttributes attributes, String aeqId, String[] fields,
+                                ExecutorService waitingThreadPool) {
+    return new LuceneRawIndex(indexName, regionPath, cache, analyzer,
+        fieldAnalyzers, serializer, attributes, aeqId, fields, waitingThreadPool);
+  }
 }
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 7313a82..9a9ce7c 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
@@ -107,6 +107,7 @@ public class LuceneRegionListener implements RegionListener {
   public void afterCreate(Region region) {
     if (region.getFullPath().equals(this.regionPath)
         && this.afterCreateInvoked.compareAndSet(false, true)) {
+      System.err.println(this+".afterCreate Service: "+service+"   LuceneIndex: "+luceneIndex);
       this.service.afterDataRegionCreated(this.luceneIndex);
     }
   }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
index 1f8184c..5451b5f 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
@@ -166,8 +166,7 @@ public class LuceneServiceImpl implements InternalLuceneService {
     if (!regionPath.startsWith("/")) {
       regionPath = "/" + regionPath;
     }
-    String name = indexName + "#" + regionPath.replace('/', '_');
-    return name;
+    return indexName + "#" + regionPath.replace('/', '_');
   }
 
   public static String getUniqueIndexRegionName(String indexName, String regionPath,
@@ -232,6 +231,7 @@ public class LuceneServiceImpl implements InternalLuceneService {
     LuceneIndexImpl luceneIndex = beforeDataRegionCreated(indexName, regionPath,
         region.getAttributes(), analyzer, fieldAnalyzers, aeqId, serializer, fields);
 
+    System.out.println(this+".createIndexOnExistingRegion  index: "+luceneIndex);
     afterDataRegionCreated(luceneIndex);
 
     region.addAsyncEventQueueId(aeqId, true);
@@ -324,15 +324,24 @@ public class LuceneServiceImpl implements InternalLuceneService {
       RegionAttributes attributes, final Analyzer analyzer,
       final Map<String, Analyzer> fieldAnalyzers, String aeqId, final LuceneSerializer serializer,
       final String... fields) {
-    return createIndexObject(indexName, regionPath, fields, analyzer, fieldAnalyzers, serializer,
-        attributes, aeqId);
+    LuceneIndexImpl
+        index =
+        createIndexObject(indexName, regionPath, fields, analyzer, fieldAnalyzers, serializer,
+            attributes, aeqId);
+    System.err.println(this+".beforeDataRegionCreated Index: "+index);
+    return index;
   }
 
   private LuceneIndexImpl createIndexObject(String indexName, String regionPath, String[] fields,
       Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers, LuceneSerializer serializer,
       RegionAttributes attributes, String aeqId) {
-    return luceneIndexFactory.create(indexName, regionPath, cache, analyzer, fieldAnalyzers,
-        serializer, attributes, aeqId, fields, dm.getWaitingThreadPool());
+    System.err.println(this+".createIndexObject LuceneIndexFactory: "+luceneIndexFactory);
+    LuceneIndexImpl
+        index =
+        luceneIndexFactory.create(indexName, regionPath, cache, analyzer, fieldAnalyzers,
+            serializer, attributes, aeqId, fields, dm.getWaitingThreadPool());
+    System.err.println(this+".createIndexObject Index: "+index);
+    return index;
   }
 
   private void registerDefinedIndex(final String indexName, final String regionPath,
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManager.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManager.java
index 9e2e4fd..658182b 100755
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManager.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManager.java
@@ -140,6 +140,7 @@ public class PartitionedRepositoryManager implements RepositoryManager {
   protected IndexRepository computeRepository(Integer bucketId, LuceneSerializer serializer,
       InternalLuceneIndex index, PartitionedRegion userRegion, IndexRepository oldRepository)
       throws IOException {
+    System.err.println(this+".computeRepository23232 index: "+index);
     return indexRepositoryFactory.computeIndexRepository(bucketId, serializer, index, userRegion,
         oldRepository);
   }
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java
index bb38ed3..e063223 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactorySpy.java
@@ -42,11 +42,15 @@ public class LuceneIndexFactorySpy extends LuceneIndexImplFactory {
 
   @Override
   public LuceneIndexImpl create(String indexName, String regionPath, InternalCache cache,
-      Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers, LuceneSerializer serializer,
-      RegionAttributes attributes, String aeqId, String[] fields,
-      ExecutorService waitingThreadPool) {
+                                Analyzer analyzer, Map<String, Analyzer> fieldAnalyzers,
+                                LuceneSerializer serializer,
+                                RegionAttributes attributes, String aeqId, String[] fields,
+                                ExecutorService waitingThreadPool) {
+
     LuceneIndexForPartitionedRegion index =
-        Mockito.spy(new ExtendedLuceneIndexForPartitionedRegion(indexName, regionPath, cache));
+        Mockito.spy(new ExtendedLuceneIndexForPartitionedRegion(indexName, regionPath, cache, analyzer, fieldAnalyzers, serializer, attributes, aeqId,
+            fields, waitingThreadPool));
+    System.err.println("LuceneIndexFactorySpy.create  "+index);
     return index;
   }
 
@@ -57,9 +61,18 @@ public class LuceneIndexFactorySpy extends LuceneIndexImplFactory {
 
   private static class ExtendedLuceneIndexForPartitionedRegion
       extends LuceneIndexForPartitionedRegion {
-    public ExtendedLuceneIndexForPartitionedRegion(final String indexName, final String regionPath,
-        final InternalCache cache) {
-      super(indexName, regionPath, cache);
+
+
+    public ExtendedLuceneIndexForPartitionedRegion(String indexName, String regionPath,
+                                                   InternalCache cache,
+                                                   Analyzer analyzer,
+                                                   Map<String, Analyzer> fieldAnalyzers,
+                                                   LuceneSerializer serializer,
+                                                   RegionAttributes attributes, String aeqId,
+                                                   String[] fields,
+                                                   ExecutorService waitingThreadPool) {
+      super(indexName, regionPath, cache, analyzer, fieldAnalyzers, serializer, attributes, aeqId,
+          fields, waitingThreadPool);
     }
 
   }
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 07c2d6c..d1e530f 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
@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -52,6 +53,7 @@ import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
 import org.apache.geode.cache.lucene.LuceneIndexFactory;
 import org.apache.geode.cache.lucene.LuceneSerializer;
 import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegionDataStore;
@@ -110,16 +112,23 @@ public class LuceneServiceImplJUnitTest {
 
   @Test
   public void userRegionShouldNotBeSetBeforeIndexInitialized() throws Exception {
+    DistributionManager dm = mock(DistributionManager.class);
     TestLuceneServiceImpl testService = new TestLuceneServiceImpl();
     Field f = LuceneServiceImpl.class.getDeclaredField("cache");
     f.setAccessible(true);
     f.set(testService, cache);
+    f = LuceneServiceImpl.class.getDeclaredField("dm");
+    f.setAccessible(true);
+    f.set(testService, dm);
     AsyncEventQueueFactoryImpl aeqFactory = mock(AsyncEventQueueFactoryImpl.class);
     when(cache.createAsyncEventQueueFactory()).thenReturn(aeqFactory);
 
     DistributedSystem ds = mock(DistributedSystem.class);
+
     Statistics luceneIndexStats = mock(Statistics.class);
     when(cache.getDistributedSystem()).thenReturn(ds);
+    when(cache.getDistributionManager()).thenReturn(dm);
+    when(dm.getWaitingThreadPool()).thenReturn(Executors.newSingleThreadExecutor());
     when(((StatisticsFactory) ds).createAtomicStatistics(any(), anyString()))
         .thenReturn(luceneIndexStats);
     when(cache.getRegion(anyString())).thenReturn(region);

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