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/18 02:34:45 UTC

incubator-geode git commit: add class LuceneIndexImpl and its subclasses and fixed TopEntriesFuncionCollectorJUnitTest

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-11 38ff05e58 -> dce9faec3


add class LuceneIndexImpl and its subclasses
and fixed TopEntriesFuncionCollectorJUnitTest


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

Branch: refs/heads/feature/GEODE-11
Commit: dce9faec31fddabebd2018ae2a1abb376b6a93af
Parents: 38ff05e
Author: zhouxh <gz...@pivotal.io>
Authored: Thu Sep 17 16:52:22 2015 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Thu Sep 17 17:24:27 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndex.java       |  22 +++-
 .../gemfire/cache/lucene/LuceneService.java     |  12 +-
 .../LuceneIndexForPartitionedRegion.java        | 114 ++++++++++++++++
 .../LuceneIndexForReplicatedRegion.java         |  52 ++++++++
 .../cache/lucene/internal/LuceneIndexImpl.java  |  66 +++++++---
 .../lucene/internal/LuceneServiceImpl.java      |  54 ++++++--
 .../distributed/TopEntriesCollectorManager.java |   2 +-
 .../internal/LuceneIndexImplJUnitTest.java      | 131 +++++++++++++++++++
 .../TopEntriesFunctionCollectorJUnitTest.java   |   2 +-
 9 files changed, 414 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dce9faec/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 9b08b05..04219ab 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
@@ -8,11 +8,16 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 
+import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
+import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
+import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
+
 
 /**
  * An lucene index is built over the data stored in a GemFire Region.
@@ -35,7 +40,7 @@ public interface LuceneIndex {
   /**
    * @return the region name for this index
    */
-  public String getRegionName();
+  public String getRegionPath();
       
   /**
    * @return the indexed field names in a Set
@@ -43,8 +48,23 @@ 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/dce9faec/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
index 45bbc48..eed80d9 100644
--- a/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
@@ -60,22 +60,22 @@ public interface LuceneService {
    * Create a lucene index using default analyzer.
    * 
    * @param indexName
-   * @param regionName
+   * @param regionPath
    * @param fields
    * @return LuceneIndex object
    */
-  public LuceneIndex createIndex(String indexName, String regionName, String... fields);
+  public LuceneIndex createIndex(String indexName, String regionPath, String... fields);
   
   /**
    * Create a lucene index using specified analyzer per field
    * 
    * @param indexName index name
-   * @param regionName region name
+   * @param regionPath region name
    * @param analyzerPerField analyzer per field map
    * @return LuceneIndex object
    *
    */
-  public LuceneIndex createIndex(String indexName, String regionName,  
+  public LuceneIndex createIndex(String indexName, String regionPath,  
       Map<String, Analyzer> analyzerPerField);
 
   /**
@@ -88,10 +88,10 @@ public interface LuceneService {
   /**
    * Get the lucene index object specified by region name and index name
    * @param indexName index name
-   * @param regionName region name
+   * @param regionPath region name
    * @return LuceneIndex object
    */
-  public LuceneIndex getIndex(String indexName, String regionName);
+  public LuceneIndex getIndex(String indexName, String regionPath);
   
   /**
    * get all the lucene indexes.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dce9faec/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
new file mode 100644
index 0000000..faddbbc
--- /dev/null
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -0,0 +1,114 @@
+package com.gemstone.gemfire.cache.lucene.internal;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionShortcut;
+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.cache.partition.PartitionRegionHelper;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+
+/* wrapper of IndexWriter */
+public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
+
+
+  Cache cache;
+  // map to map bucketId to repo, -1 means its DR
+  HashMap<Integer, IndexRepository> indexRepositories = new HashMap<Integer, IndexRepository>();
+
+  public LuceneIndexForPartitionedRegion(String indexName, String regionPath, Cache cache) {
+    this.indexName = indexName;
+    this.regionPath = regionPath;
+    this.cache = cache;
+    initialize();
+  }
+  
+  public void initialize() {
+    if (!hasInitialized) {
+      /* create index region */
+      PartitionedRegion dataRegion = (PartitionedRegion)cache.getRegion(regionPath);
+      assert dataRegion != null;
+      RegionAttributes ra = dataRegion.getAttributes();
+      DataPolicy dp = ra.getDataPolicy();
+      final boolean isPartitionedRegion = (ra.getPartitionAttributes() == null) ? false : true;
+      final boolean withPersistence = dp.withPersistence();
+      final boolean withStorage = isPartitionedRegion?ra.getPartitionAttributes().getLocalMaxMemory()>0:dp.withStorage();
+      RegionShortcut regionShortCut;
+      if (isPartitionedRegion) {
+        if (withPersistence) {
+          // TODO: add PartitionedRegionAttributes instead
+          regionShortCut = RegionShortcut.PARTITION_PERSISTENT;
+        } else {
+          regionShortCut = RegionShortcut.PARTITION;
+        }
+      } else {
+        if (withPersistence) {
+          regionShortCut = RegionShortcut.REPLICATE_PERSISTENT;
+        } else {
+          regionShortCut = RegionShortcut.REPLICATE;
+        }
+      }
+
+      // final boolean isOffHeap = ra.getOffHeap();
+
+      // TODO: 1) dataRegion should be withStorage
+      //       2) Persistence to Persistence
+      //       3) Replicate to Replicate, Partition To Partition
+      //       4) Offheap to Offheap
+      if (!withStorage) {
+        throw new IllegalStateException("The data region to create lucene index should be with storage");
+      }
+
+      // create PR fileRegion, but not to create its buckets for now
+      final String fileRegionName = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath)+".files";
+      fileRegion = cache.<String, File> getRegion(fileRegionName);
+      if (null == fileRegion) {
+        fileRegion = cache.<String, File> createRegionFactory(regionShortCut)
+            .setPartitionAttributes(new PartitionAttributesFactory<String, File>().setColocatedWith(regionPath)
+                .create())
+                .create(fileRegionName);
+      }
+
+      // create PR chunkRegion, but not to create its buckets for now
+      final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
+      chunkRegion = cache.<ChunkKey, byte[]> getRegion(chunkRegionName);
+      if (null == chunkRegion) {
+        chunkRegion = cache.<ChunkKey, byte[]> createRegionFactory(regionShortCut)
+            .setPartitionAttributes(new PartitionAttributesFactory<ChunkKey, byte[]>().setColocatedWith(fileRegionName)
+                .create())
+                .create(chunkRegionName);
+      }
+
+      // we will create RegionDirectorys on the fly when data coming
+
+      hasInitialized = true;
+    }
+  }
+
+  public void close() {
+    // TODO Auto-generated method stub
+    
+  }
+  
+  @Override
+  public Collection<IndexRepository> getRepository(RegionFunctionContext ctx) {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dce9faec/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
new file mode 100644
index 0000000..8f93947
--- /dev/null
+++ b/gemfire-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForReplicatedRegion.java
@@ -0,0 +1,52 @@
+package com.gemstone.gemfire.cache.lucene.internal;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+
+import com.gemstone.gemfire.cache.Cache;
+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;
+
+/* wrapper of IndexWriter */
+public class LuceneIndexForReplicatedRegion extends LuceneIndexImpl {
+
+  IndexRepository indexRepository = null;
+
+  public LuceneIndexForReplicatedRegion(String indexName, String regionPath, Cache cache) {
+    // TODO Auto-generated constructor stub
+  }
+
+  public void initialize() {
+    // TODO Auto-generated method stub
+    
+  }
+
+  public void close() {
+    // TODO Auto-generated method stub
+    
+  }
+
+  @Override
+  public Map<String, Analyzer> getFieldAnalyzerMap() {
+    // TODO Auto-generated method stub
+    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/dce9faec/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 a7bdd99..9741775 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,53 +1,77 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 
+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;
 
-/* wrapper of IndexWriter */
-public class LuceneIndexImpl implements LuceneIndex {
+public abstract class LuceneIndexImpl implements LuceneIndex {
 
+  static private final boolean CREATE_CACHE = Boolean.getBoolean("lucene.createCache");
+  static private final boolean USE_FS = Boolean.getBoolean("lucene.useFileSystem");
+  
+  protected HashSet<String> searchableFieldNames = new HashSet<String>();
+  protected HashSet<String> searchablePDXFieldNames = new HashSet<String>();
+  
   /* searchable fields should belong to a specific index
    */
-  HashSet<String> searchableFieldNames;
-  
-  HashSet<String> searchablePDXFieldNames;
+  Region<String, File> fileRegion;
+  Region<ChunkKey, byte[]> chunkRegion;
   
+  protected String indexName;
+  protected String regionPath;
+  protected boolean hasInitialized = false;
+
   @Override
   public String getName() {
-    // TODO Auto-generated method stub
-    return null;
+    return this.indexName;
   }
 
   @Override
-  public String getRegionName() {
-    // TODO Auto-generated method stub
-    return null;
+  public String getRegionPath() {
+    return this.regionPath;
+  }
+  
+  protected void addSearchableField(String field) {
+    searchableFieldNames.add(field);
+  }
+  
+  protected void addSearchablePDXField(String field) {
+    searchablePDXFieldNames.add(field);
   }
 
   @Override
   public String[] getFieldNames() {
-    // TODO Auto-generated method stub
-    return null;
+    String[] fieldNames = new String[searchableFieldNames.size()];
+    return searchableFieldNames.toArray(fieldNames);
   }
 
-  public void initialize() {
-    // TODO Auto-generated method stub
-    
+  @Override
+  public String[] getPDXFieldNames() {
+    String[] pdxFieldNames = new String[searchablePDXFieldNames.size()];;
+    return searchablePDXFieldNames.toArray(pdxFieldNames);
   }
-
-  public void close() {
+  
+  @Override
+  public Map<String, Analyzer> getFieldAnalyzerMap() {
     // TODO Auto-generated method stub
-    
+    // Will do that later: Gester
+    return null;
   }
 
   @Override
-  public Map<String, Analyzer> getFieldAnalyzerMap() {
-    // TODO Auto-generated method stub
+  public Collection<IndexRepository> getRepository(RegionFunctionContext ctx) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dce9faec/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 47e6424..245287c 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
@@ -8,11 +8,14 @@ import org.apache.lucene.analysis.Analyzer;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.extension.Extensible;
 import com.gemstone.gemfire.internal.cache.extension.Extension;
 import com.gemstone.gemfire.internal.cache.xmlcache.XmlGenerator;
@@ -44,21 +47,40 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> {
     // Initialize the Map which maintains indexes
     this.indexMap = new HashMap<String, LuceneIndex>();
   }
-
-  public String getUniqueIndexName(String indexName, String regionName) {
-    String name = indexName + "#" + regionName.replace('/', '_');
+  
+  public static String getUniqueIndexName(String indexName, String regionPath) {
+    String name = indexName + "#" + regionPath.replace('/', '_');
     return name;
   }
 
   @Override
-  public LuceneIndex createIndex(String indexName, String regionName, String... fields) {
-    // TODO Auto-generated method stub
-    return null;
+  public LuceneIndex createIndex(String indexName, String regionPath, String... fields) {
+    Region dataregion = this.cache.getRegion(regionPath);
+    if (dataregion == null) {
+      cache.getLogger().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;
   }
 
   @Override
-  public LuceneIndex getIndex(String indexName, String regionName) {
-    return indexMap.get(getUniqueIndexName(indexName, regionName));
+  public LuceneIndex getIndex(String indexName, String regionPath) {
+    return indexMap.get(getUniqueIndexName(indexName, regionPath));
   }
 
   @Override
@@ -67,7 +89,7 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> {
   }
 
   @Override
-  public LuceneIndex createIndex(String indexName, String regionName, Map<String, Analyzer> analyzerPerField) {
+  public LuceneIndex createIndex(String indexName, String regionPath, Map<String, Analyzer> analyzerPerField) {
     // TODO Auto-generated method stub
     return null;
   }
@@ -75,8 +97,8 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> {
   @Override
   public void destroyIndex(LuceneIndex index) {
     LuceneIndexImpl indexImpl = (LuceneIndexImpl) index;
-    indexMap.remove(getUniqueIndexName(index.getName(), index.getRegionName()));
-    indexImpl.close();
+    indexMap.remove(getUniqueIndexName(index.getName(), index.getRegionPath()));
+//    indexImpl.close();
   }
 
   @Override
@@ -95,5 +117,15 @@ public class LuceneServiceImpl implements LuceneService, Extension<Cache> {
     // TODO Auto-generated method stub
 
   }
+  
+  public void registerIndex(final String regionAndIndex, LuceneIndex index){
+    if( !indexMap.containsKey( regionAndIndex )) {
+      indexMap.put(regionAndIndex, index);
+    }
+  }
+
+  public void unregisterIndex(final String region){
+    if( indexMap.containsKey( region )) indexMap.remove( region );
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dce9faec/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 f269b2b..bf7a265 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
@@ -60,7 +60,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>>(Collections.reverseOrder(entryListComparator));
+    entryListsPriorityQueue = new PriorityQueue<List<EntryScore>>(11, Collections.reverseOrder(entryListComparator));
     TopEntriesCollector mergedResult = new TopEntriesCollector(id, limit);
 
     for (IndexResultCollector collector : collectors) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dce9faec/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
new file mode 100644
index 0000000..5dcaf9f
--- /dev/null
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
@@ -0,0 +1,131 @@
+package com.gemstone.gemfire.cache.lucene.internal;
+
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.apache.lucene.queryparser.classic.QueryParser;
+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.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
+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.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.Type2;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+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.
+ */
+@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/dce9faec/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 d45865d..6e02e4e 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());
-    throw exception.get();
+    if (exception != null) throw exception.get();
   }
 
   @Test