You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ja...@apache.org on 2016/04/22 00:51:56 UTC

incubator-geode git commit: GEODE-1188: Lucene indexes can be created with partition redundant regions

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 7de1d32b1 -> a194b03fc


GEODE-1188: Lucene indexes can be created with partition redundant regions

Added a test to test against all partition region types, not including proxy types

When specific region types need more robust/specific tests, we can add specific tests
for the type and eventually remove this test if/when all types are covered


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

Branch: refs/heads/develop
Commit: a194b03fc9a8a0e4691cb62d008a3dc3026ae5e7
Parents: 7de1d32
Author: Jason Huynh <hu...@gmail.com>
Authored: Wed Apr 20 15:24:07 2016 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Thu Apr 21 15:50:22 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexForPartitionedRegion.java        |  30 ++--
 .../internal/LuceneServiceImplJUnitTest.java    | 146 +++++++++++++------
 2 files changed, 116 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a194b03f/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index 4145459..0d257e3 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -138,13 +138,14 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
 
   Region createFileRegion(final RegionShortcut regionShortCut,
                                 final String fileRegionName,
-                                final PartitionAttributes partitionAttributes)
-  {
+                                final PartitionAttributes partitionAttributes) {
+    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory<String, File>();
+    partitionAttributesFactory.setColocatedWith(regionPath);
+    configureLuceneRegionAttributesFactory(partitionAttributesFactory, partitionAttributes);
+
     return cache.<String, File> createRegionFactory(regionShortCut)
-        .setPartitionAttributes(new PartitionAttributesFactory<String, File>().setColocatedWith(regionPath)
-            .setTotalNumBuckets(partitionAttributes.getTotalNumBuckets())
-            .create())
-            .create(fileRegionName);
+        .setPartitionAttributes(partitionAttributesFactory.create())
+        .create(fileRegionName);
   }
 
   String createFileRegionName() {
@@ -157,12 +158,13 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
 
   Region<ChunkKey, byte[]> createChunkRegion(final RegionShortcut regionShortCut,
                            final String fileRegionName,
-                           final PartitionAttributes partitionAttributes, final String chunkRegionName)
-  {
+                           final PartitionAttributes partitionAttributes, final String chunkRegionName) {
+    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory<String, File>();
+    partitionAttributesFactory.setColocatedWith(fileRegionName);
+    configureLuceneRegionAttributesFactory(partitionAttributesFactory, partitionAttributes);
+
     return cache.<ChunkKey, byte[]> createRegionFactory(regionShortCut)
-      .setPartitionAttributes(new PartitionAttributesFactory<ChunkKey, byte[]>().setColocatedWith(fileRegionName)
-        .setTotalNumBuckets(partitionAttributes.getTotalNumBuckets())
-        .create())
+      .setPartitionAttributes(partitionAttributesFactory.create())
       .create(chunkRegionName);
   }
 
@@ -170,7 +172,11 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
   }
 
-
+  private PartitionAttributesFactory configureLuceneRegionAttributesFactory(PartitionAttributesFactory attributesFactory, PartitionAttributes dataRegionAttributes) {
+    attributesFactory.setTotalNumBuckets(dataRegionAttributes.getTotalNumBuckets());
+    attributesFactory.setRedundantCopies(dataRegionAttributes.getRedundantCopies());
+    return attributesFactory;
+  }
 
   public void close() {
     // TODO Auto-generated method stub

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a194b03f/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
index a85e617..1beea0f 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
@@ -26,6 +26,8 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import com.gemstone.gemfire.cache.Region;
+
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
@@ -63,6 +65,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class LuceneServiceImplJUnitTest {
+
   Cache cache;
   ClientCache clientCache;
   private LuceneIndexImpl repo;
@@ -96,6 +99,13 @@ public class LuceneServiceImplJUnitTest {
   }
 
   @After
+  public void destroyService() {
+    if (null != service) {
+      service = null;
+    }
+  }
+
+  @After
   public void destroyCache() {
     if (null != cache && !cache.isClosed()) {
       cache.close();
@@ -134,28 +144,34 @@ public class LuceneServiceImplJUnitTest {
     return service;
   }
 
+
+  private Region createRegion(String regionName, RegionShortcut shortcut) {
+    return cache.createRegionFactory(shortcut).create(regionName);
+  }
+
   private LocalRegion createPR(String regionName, boolean isSubRegion) {
     if (isSubRegion) {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION).create("root");
-      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
+      LocalRegion root = (LocalRegion) cache.createRegionFactory(RegionShortcut.PARTITION).create("root");
+      LocalRegion region = (LocalRegion) cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
         createSubregion(root, regionName);
       return region;
-    } else {
-      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
-        create(regionName);
+    }
+    else {
+      LocalRegion region = (LocalRegion) createRegion(regionName, RegionShortcut.PARTITION_PERSISTENT);
       return region;
     }
   }
 
   private LocalRegion createRR(String regionName, boolean isSubRegion) {
     if (isSubRegion) {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
-      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
+
+      LocalRegion root = (LocalRegion) cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
+      LocalRegion region = (LocalRegion) cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
         createSubregion(root, regionName);
       return region;
-    } else {
-      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
-        create(regionName);
+    }
+    else {
+      LocalRegion region = (LocalRegion) createRegion(regionName, RegionShortcut.REPLICATE_PERSISTENT);
       return region;
     }
   }
@@ -169,38 +185,6 @@ public class LuceneServiceImplJUnitTest {
   }
 
   @Test
-  public void canCreateLuceneIndexForPR() throws IOException, ParseException {
-    getService();
-    service.createIndex("index1", "PR1", "field1", "field2", "field3");
-    LocalRegion userRegion = createPR("PR1", false);
-    LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
-    assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
-    LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion)index1;
-    assertEquals("index1", index1.getName());
-    assertEquals("/PR1", index1.getRegionPath());
-    String[] fields1 = index1.getFieldNames();
-    assertEquals(3, fields1.length);
-    Analyzer analyzer = index1PR.getAnalyzer();
-    assertTrue(analyzer instanceof StandardAnalyzer);
-    RepositoryManager RepositoryManager = index1PR.getRepositoryManager();
-    assertTrue(RepositoryManager != null);
-
-    final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1")+".files";
-    final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1")+".chunks";
-    PartitionedRegion filePR = (PartitionedRegion)cache.getRegion(fileRegionName);
-    PartitionedRegion chunkPR = (PartitionedRegion)cache.getRegion(chunkRegionName);
-    assertTrue(filePR != null);
-    assertTrue(chunkPR != null);
-
-    String aeqId = LuceneServiceImpl.getUniqueIndexName(index1.getName(), index1.getRegionPath());
-    AsyncEventQueueImpl aeq = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
-    assertTrue(aeq != null);
-
-    //Make sure our queue doesn't show up in the list of async event queues 
-    assertEquals(Collections.emptySet(), cache.getAsyncEventQueues());
-  }
-
-  @Test
   public void canCreateLuceneIndexForPRWithAnalyzer() throws IOException, ParseException {
     getService();
     StandardAnalyzer sa = new StandardAnalyzer();
@@ -214,9 +198,9 @@ public class LuceneServiceImplJUnitTest {
 
     service.createIndex("index1", "PR1", analyzerPerField);
     createPR("PR1", false);
-    LuceneIndexImpl index1 = (LuceneIndexImpl)service.getIndex("index1", "PR1");
+    LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
     assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
-    LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion)index1;
+    LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion) index1;
     assertEquals("index1", index1.getName());
     assertEquals("/PR1", index1.getRegionPath());
     String[] fields1 = index1.getFieldNames();
@@ -226,10 +210,10 @@ public class LuceneServiceImplJUnitTest {
     RepositoryManager RepositoryManager = index1PR.getRepositoryManager();
     assertTrue(RepositoryManager != null);
 
-    final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1")+".files";
-    final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1")+".chunks";
-    PartitionedRegion filePR = (PartitionedRegion)cache.getRegion(fileRegionName);
-    PartitionedRegion chunkPR = (PartitionedRegion)cache.getRegion(chunkRegionName);
+    final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".files";
+    final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".chunks";
+    PartitionedRegion filePR = (PartitionedRegion) cache.getRegion(fileRegionName);
+    PartitionedRegion chunkPR = (PartitionedRegion) cache.getRegion(chunkRegionName);
     assertTrue(filePR != null);
     assertTrue(chunkPR != null);
   }
@@ -243,4 +227,70 @@ public class LuceneServiceImplJUnitTest {
     createRR("RR1", false);
   }
 
+  @Test
+  public void canCreateIndexForAllNonProxyPartitionRegionTypes() {
+    for (RegionShortcut shortcut : RegionShortcut.values()) {
+      String sname = shortcut.name().toLowerCase();
+      if (sname.contains("partition") && !sname.contains("proxy")) {
+        canCreateLuceneIndexForPRType(shortcut);
+        //Destroying cache and service for now because aeq's are not completely being cleaned up correctly after
+        // being destroyed.  Instead we should close the aeq and clean up any regions associated with this lucene
+        //index but only after aeq destroy works properly
+        destroyCache();
+        destroyService();
+      }
+    }
+  }
+
+  public void canCreateLuceneIndexForPRType(RegionShortcut regionShortcut) {
+    getService();
+    service.createIndex("index1", "PR1", "field1", "field2", "field3");
+    Region region = null;
+    AsyncEventQueueImpl aeq = null;
+    try {
+      region = createRegion("PR1", regionShortcut);
+      LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
+      assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
+      LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion) index1;
+      assertEquals("index1", index1.getName());
+      assertEquals("/PR1", index1.getRegionPath());
+      String[] fields1 = index1.getFieldNames();
+      assertEquals(3, fields1.length);
+      Analyzer analyzer = index1PR.getAnalyzer();
+      assertTrue(analyzer instanceof StandardAnalyzer);
+      RepositoryManager RepositoryManager = index1PR.getRepositoryManager();
+      assertTrue(RepositoryManager != null);
+
+      final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".files";
+      final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".chunks";
+      PartitionedRegion filePR = (PartitionedRegion) cache.getRegion(fileRegionName);
+      PartitionedRegion chunkPR = (PartitionedRegion) cache.getRegion(chunkRegionName);
+      assertTrue(filePR != null);
+      assertTrue(chunkPR != null);
+
+      String aeqId = LuceneServiceImpl.getUniqueIndexName(index1.getName(), index1.getRegionPath());
+      aeq = (AsyncEventQueueImpl) cache.getAsyncEventQueue(aeqId);
+      assertTrue(aeq != null);
+
+      //Make sure our queue doesn't show up in the list of async event queues
+      assertEquals(Collections.emptySet(), cache.getAsyncEventQueues());
+    }
+    finally {
+      String aeqId = LuceneServiceImpl.getUniqueIndexName("index1", "PR1");
+      PartitionedRegion chunkRegion = (PartitionedRegion) cache.getRegion(aeqId + ".chunks");
+      if (chunkRegion != null) {
+        chunkRegion.destroyRegion();
+      }
+      PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
+      if (fileRegion != null) {
+        fileRegion.destroyRegion();
+      }
+      ((GemFireCacheImpl) cache).removeAsyncEventQueue(aeq);
+      if (aeq != null) {
+        aeq.destroy();
+      }
+      region.destroyRegion();
+    }
+  }
+
 }