You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by zh...@apache.org on 2015/09/23 18:31:15 UTC

incubator-geode git commit: add analyzer to luceneIndexImpl add RepositoryManager into luceneIndexImpl

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-11 c80e93b95 -> 9ca4e784a


add analyzer to luceneIndexImpl
add RepositoryManager into luceneIndexImpl


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

Branch: refs/heads/feature/GEODE-11
Commit: 9ca4e784adec71acad6cac299229cfa493737608
Parents: c80e93b
Author: zhouxh <gz...@pivotal.io>
Authored: Mon Sep 21 17:41:22 2015 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Wed Sep 23 09:30:01 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndex.java       | 15 ---
 .../LuceneIndexForPartitionedRegion.java        | 10 +-
 .../LuceneIndexForReplicatedRegion.java         | 10 +-
 .../cache/lucene/internal/LuceneIndexImpl.java  | 42 ++++-----
 .../lucene/internal/LuceneServiceImpl.java      | 45 +++++++--
 .../distributed/TopEntriesCollectorManager.java |  2 +-
 .../internal/LuceneIndexImplJUnitTest.java      | 93 +-----------------
 .../internal/LuceneServiceImplJUnitTest.java    | 99 +++++++++++++++++++-
 .../TopEntriesFunctionCollectorJUnitTest.java   |  2 +-
 9 files changed, 162 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
index 04219ab..328eed8 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
@@ -48,23 +48,8 @@ public interface LuceneIndex {
   public String[] getFieldNames();
   
   /**
-   * @return the indexed PDX field names in a Set
-   */
-  public String[] getPDXFieldNames();
-  
-  /**
    * @return the field to analyzer map
    */
   public Map<String, Analyzer> getFieldAnalyzerMap();
   
-  /**
-   * Returns a collection of {@link IndexRepository} instances hosting index data of the input list of bucket ids. The
-   * bucket needs to be present on this member.
-   * 
-   * @param ctx {@link RegionFunctionContext} function context. It's either a replicated region
-   * or local buckets of a Partitioned region for which {@link IndexRepository}s needs to be discovered. 
-   * empty for all primary buckets are not on this member.
-   * @return a collection of {@link IndexRepository} instances
-   */
-  public Collection<IndexRepository> getRepository(RegionFunctionContext ctx);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index f648c9a..cbab401 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -20,6 +20,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.LuceneSerializer;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -33,9 +34,9 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     this.indexName = indexName;
     this.regionPath = regionPath;
     this.cache = cache;
-    initialize();
   }
   
+  @Override
   public void initialize() {
     if (!hasInitialized) {
       /* create index region */
@@ -93,7 +94,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       }
 
       // we will create RegionDirectorys on the fly when data coming
-
+      repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion)fileRegion, (PartitionedRegion)chunkRegion, null, analyzer);
       hasInitialized = true;
     }
   }
@@ -103,9 +104,4 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     
   }
   
-  @Override
-  public Collection<IndexRepository> getRepository(RegionFunctionContext ctx) {
-    return null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
index 8f93947..f160a6d 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
@@ -4,6 +4,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -20,8 +21,6 @@ import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 /* wrapper of IndexWriter */
 public class LuceneIndexForReplicatedRegion extends LuceneIndexImpl {
 
-  IndexRepository indexRepository = null;
-
   public LuceneIndexForReplicatedRegion(String indexName, String regionPath, Cache cache) {
     // TODO Auto-generated constructor stub
   }
@@ -42,11 +41,4 @@ public class LuceneIndexForReplicatedRegion extends LuceneIndexImpl {
     return null;
   }
 
-  @Override
-  public Collection<IndexRepository> getRepository(RegionFunctionContext ctx) {
-    ArrayList<IndexRepository> repositories = new ArrayList<IndexRepository>();
-    repositories.add(indexRepository);
-    return repositories;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
index 9741775..799ef27 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
@@ -1,20 +1,16 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
-import java.util.Collection;
 import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
-import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 
 public abstract class LuceneIndexImpl implements LuceneIndex {
 
@@ -22,10 +18,9 @@ public abstract class LuceneIndexImpl implements LuceneIndex {
   static private final boolean USE_FS = Boolean.getBoolean("lucene.useFileSystem");
   
   protected HashSet<String> searchableFieldNames = new HashSet<String>();
-  protected HashSet<String> searchablePDXFieldNames = new HashSet<String>();
+  protected RepositoryManager repositoryManager;
+  protected Analyzer analyzer;
   
-  /* searchable fields should belong to a specific index
-   */
   Region<String, File> fileRegion;
   Region<ChunkKey, byte[]> chunkRegion;
   
@@ -47,10 +42,6 @@ public abstract class LuceneIndexImpl implements LuceneIndex {
     searchableFieldNames.add(field);
   }
   
-  protected void addSearchablePDXField(String field) {
-    searchablePDXFieldNames.add(field);
-  }
-
   @Override
   public String[] getFieldNames() {
     String[] fieldNames = new String[searchableFieldNames.size()];
@@ -58,21 +49,28 @@ public abstract class LuceneIndexImpl implements LuceneIndex {
   }
 
   @Override
-  public String[] getPDXFieldNames() {
-    String[] pdxFieldNames = new String[searchablePDXFieldNames.size()];;
-    return searchablePDXFieldNames.toArray(pdxFieldNames);
-  }
-  
-  @Override
   public Map<String, Analyzer> getFieldAnalyzerMap() {
     // TODO Auto-generated method stub
     // Will do that later: Gester
     return null;
   }
 
-  @Override
-  public Collection<IndexRepository> getRepository(RegionFunctionContext ctx) {
-    return null;
+  public RepositoryManager getRepositoryManager() {
+    return this.repositoryManager;
+  }
+  
+  public void setAnalyzer(Analyzer analyzer) {
+    if (analyzer == null) {
+      this.analyzer = new StandardAnalyzer();
+    } else {
+      this.analyzer = analyzer;
+    }
   }
 
+  public Analyzer getAnalyzer() {
+    return this.analyzer;
+  }
+
+  protected void initialize() {
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
index 0f10dac..1d7178a 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
@@ -4,7 +4,10 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
@@ -28,6 +31,7 @@ import com.gemstone.gemfire.internal.cache.extension.Extensible;
 import com.gemstone.gemfire.internal.cache.extension.Extension;
 import com.gemstone.gemfire.internal.cache.xmlcache.XmlGenerator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
 
 /**
  * Implementation of LuceneService to create lucene index and query.
@@ -40,6 +44,8 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> {
   private final Cache cache;
 
   private final HashMap<String, LuceneIndex> indexMap;
+  
+  private static final Logger logger = LogService.getLogger();
 
   public LuceneServiceImpl(final Cache cache) {
     if (cache == null) {
@@ -64,26 +70,34 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> {
 
   @Override
   public LuceneIndex createIndex(String indexName, String regionPath, String... fields) {
+    LuceneIndexImpl index = createIndexRegions(indexName, regionPath);
+    if (index == null) {
+      return null;
+    }
+    for (String field:fields) {
+      index.addSearchableField(field);
+    }
+    // for this API, set index to use the default StandardAnalyzer for each field
+    index.setAnalyzer(null);
+    index.initialize();
+    registerIndex(getUniqueIndexName(indexName, regionPath), index);
+    return index;
+  }
+  
+  private LuceneIndexImpl createIndexRegions(String indexName, String regionPath) {
     Region dataregion = this.cache.getRegion(regionPath);
     if (dataregion == null) {
-      cache.getLogger().info("Data region "+regionPath+" not found");
+      logger.info("Data region "+regionPath+" not found");
       return null;
     }
     LuceneIndexImpl index = null;
     if (dataregion instanceof PartitionedRegion) {
       // partitioned region
       index = new LuceneIndexForPartitionedRegion(indexName, regionPath, cache);
-      for (String field:fields) {
-        index.addSearchableField(field);
-        index.addSearchablePDXField(field);
-      }
-      registerIndex(getUniqueIndexName(indexName, regionPath), index);
     } else {
       // replicated region
       index = new LuceneIndexForReplicatedRegion(indexName, regionPath, cache);
-      registerIndex(getUniqueIndexName(indexName, regionPath), index);
     }
-    // TODO add fields
     return index;
   }
 
@@ -99,8 +113,19 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> {
 
   @Override
   public LuceneIndex createIndex(String indexName, String regionPath, Map<String, Analyzer> analyzerPerField) {
-    // TODO Auto-generated method stub
-    return null;
+    LuceneIndexImpl index = createIndexRegions(indexName, regionPath);
+    if (index == null) {
+      return null;
+    }
+    
+    Analyzer analyzer = new PerFieldAnalyzerWrapper(new StandardAnalyzer(), analyzerPerField);
+    for (String field:analyzerPerField.keySet()) {
+      index.addSearchableField(field);
+    }
+    index.setAnalyzer(analyzer);
+    index.initialize();
+    registerIndex(getUniqueIndexName(indexName, regionPath), index);
+    return index;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorManager.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorManager.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorManager.java
index a0a209a..a5e891d 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorManager.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorManager.java
@@ -65,7 +65,7 @@ public class TopEntriesCollectorManager implements CollectorManager<TopEntriesCo
     // The queue contains iterators for all bucket results. The queue puts the entry with the highest score at the head
     // using score comparator.
     PriorityQueue<List<EntryScore>> entryListsPriorityQueue;
-    entryListsPriorityQueue = new PriorityQueue<List<EntryScore>>(11, Collections.reverseOrder(entryListComparator));
+    entryListsPriorityQueue = new PriorityQueue<List<EntryScore>>(Collections.reverseOrder(entryListComparator));
     TopEntriesCollector mergedResult = new TopEntriesCollector(id, limit);
 
     for (IndexResultCollector collector : collectors) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
index 5dcaf9f..789f243 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
@@ -34,98 +34,11 @@ import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
- * Test of the {@link IndexRepository} and everything below
- * it. This tests that we can save gemfire objects or PDXInstance
- * objects into a lucene index and search for those objects later.
+ * Test of the {@link LuceneIndexImpl} and everything below
+ * it. This tests creating lucene index with index regions and specify fields 
+ * and analyzer.
  */
 @Category(IntegrationTest.class)
 public class LuceneIndexImplJUnitTest {
 
-  private LuceneIndexImpl repo;
-  private HeterogenousLuceneSerializer mapper;
-  private StandardAnalyzer analyzer = new StandardAnalyzer();
-  private IndexWriter writer;
-
-  Cache cache = null;
-  LuceneServiceImpl service = null;
-  
-  @Before
-  public void setUp() throws IOException {
-    if (cache == null) {
-      getCache();
-    }
-    if (service == null) {
-      service = (LuceneServiceImpl)LuceneServiceProvider.get(cache);
-    }
-    
-  }
-  
-  private void getCache() {
-    try {
-       cache = CacheFactory.getAnyInstance();
-    } catch (Exception e) {
-      //ignore
-    }
-    if (null == cache) {
-      cache = new CacheFactory().set("mcast-port", "0").set("log-level", "error").create();
-      cache.getLogger().info("Created cache in test");
-    }
-  }
-  
-  private LocalRegion createPR(String regionName, boolean isSubRegion) {
-    if (isSubRegion) {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
-      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
-          createSubregion(root, regionName);
-      return region;
-    } else {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
-      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
-          create(regionName);
-      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).
-          createSubregion(root, regionName);
-      return region;
-    } else {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
-      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
-          create(regionName);
-      return region;
-    }
-  }
-
-  @Test
-  public void testCreateIndexForPR() throws IOException, ParseException {
-    createPR("PR1", false);
-    LuceneIndexImpl index1 = (LuceneIndexImpl)service.createIndex("index1", "PR1", "field1", "field2", "field3");
-    assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
-    LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion)index1;
-    assertEquals("index1", index1.getName());
-    assertEquals("PR1", index1.getRegionPath());
-    String[] fields1 = index1.getFieldNames();
-    String[] pdxfields1 = index1.getPDXFieldNames();
-    assertEquals(3, fields1.length);
-    assertEquals(3, pdxfields1.length);
-   
-    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);
-  }
-
-  @Test
-  public void testCreateIndexForRR() throws IOException, ParseException {
-//    service.createIndex("index1", "RR1", "field1", "field2", "field3");
-  
-    
-  }
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
index 2b890db..65fad5d 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
@@ -1,24 +1,45 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
+
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class LuceneServiceImplJUnitTest {
   Cache cache;
-
+  private LuceneIndexImpl repo;
+  private HeterogenousLuceneSerializer mapper;
+  private StandardAnalyzer analyzer = new StandardAnalyzer();
+  private IndexWriter writer;
+  LuceneServiceImpl service = null;
+  private static final Logger logger = LogService.getLogger();
+  
   // lucene service will register query execution function on initialization
   @Test
   public void shouldRegisterQueryFunction() {
@@ -43,4 +64,80 @@ public class LuceneServiceImplJUnitTest {
       cache = null;
     }
   }
+  
+  private void getCache() {
+    try {
+       cache = CacheFactory.getAnyInstance();
+    } catch (Exception e) {
+      //ignore
+    }
+    if (null == cache) {
+      cache = createBasicCache();
+    }
+  }
+  
+  private void getService() {
+    if (cache == null) {
+      getCache();
+    }
+    if (service == null) {
+      service = (LuceneServiceImpl)LuceneServiceProvider.get(cache);
+    }
+  }
+  
+  private LocalRegion createPR(String regionName, boolean isSubRegion) {
+    if (isSubRegion) {
+      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
+      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
+          createSubregion(root, regionName);
+      return region;
+    } else {
+      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
+      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
+          create(regionName);
+      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).
+          createSubregion(root, regionName);
+      return region;
+    } else {
+      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
+      LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
+          create(regionName);
+      return region;
+    }
+  }
+
+  @Test
+  public void testCreateIndexForPR() throws IOException, ParseException {
+    getService();
+    createPR("PR1", false);
+    LuceneIndexImpl index1 = (LuceneIndexImpl)service.createIndex("index1", "PR1", "field1", "field2", "field3");
+    assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
+    LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion)index1;
+    assertEquals("index1", index1.getName());
+    assertEquals("PR1", index1.getRegionPath());
+    String[] fields1 = index1.getFieldNames();
+    assertEquals(3, fields1.length);
+   
+    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);
+  }
+
+  @Test
+  public void testCreateIndexForRR() throws IOException, ParseException {
+//    service.createIndex("index1", "RR1", "field1", "field2", "field3");
+  
+    
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ca4e784/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
index c139e9f..a66f2fc 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
@@ -179,7 +179,7 @@ public class TopEntriesFunctionCollectorJUnitTest {
 
     endGetResult.await(1, TimeUnit.SECONDS);
     assertEquals(0, endGetResult.getCount());
-    if (exception != null) throw exception.get();
+    throw exception.get();
   }
 
   @Test