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 2016/05/24 22:56:40 UTC

incubator-geode git commit: GEODE-11: let query to use index's analyzer; add tests for customized analyzer and analyzer per field

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 46eeb39ce -> 777c42ee0


GEODE-11: let query to use index's analyzer; add tests for customized analyzer and analyzer per field


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

Branch: refs/heads/develop
Commit: 777c42ee08866caf63c39a20072c7930bbbd2fc5
Parents: 46eeb39
Author: zhouxh <gz...@pivotal.io>
Authored: Tue May 24 15:52:09 2016 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Tue May 24 15:55:22 2016 -0700

----------------------------------------------------------------------
 .../lucene/internal/StringQueryProvider.java    |   7 +-
 .../repository/IndexRepositoryImpl.java         |   8 ++
 .../HeterogeneousLuceneSerializer.java          |   7 ++
 .../serializer/PdxLuceneSerializer.java         |   7 ++
 .../serializer/ReflectionLuceneSerializer.java  |   8 ++
 .../cache/lucene/LuceneIntegrationTest.java     |   1 +
 .../lucene/LuceneQueriesIntegrationTest.java    | 103 ++++++++++++++++---
 .../internal/StringQueryProviderJUnitTest.java  |   8 +-
 .../distributed/LuceneFunctionJUnitTest.java    |  11 +-
 .../cache/lucene/test/LuceneTestUtilities.java  |   2 +-
 10 files changed, 139 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
index 1e2b63d..62cb65c 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
@@ -65,10 +65,13 @@ public class StringQueryProvider implements LuceneQueryProvider, DataSerializabl
     if (luceneQuery == null) {
       String[] fields = index.getFieldNames();
 
-      //TODO  get the analyzer from the index
-      MultiFieldQueryParser parser = new MultiFieldQueryParser(fields, new StandardAnalyzer());
+      LuceneIndexImpl indexImpl = (LuceneIndexImpl)index;
+      MultiFieldQueryParser parser = new MultiFieldQueryParser(fields, indexImpl.getAnalyzer());
       try {
         luceneQuery = parser.parse(query);
+        if (logger.isDebugEnabled()) {
+          logger.debug("User query "+query+" is parsed to be: "+luceneQuery);
+        }
       } catch (ParseException e) {
         logger.debug("Malformed lucene query: " + query, e);
         throw new QueryException(e);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
index e589ef4..065cc6a 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
@@ -20,7 +20,9 @@
 package com.gemstone.gemfire.cache.lucene.internal.repository;
 
 import java.io.IOException;
+import java.util.Iterator;
 
+import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
@@ -33,6 +35,7 @@ import org.apache.lucene.search.TopDocs;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.LuceneSerializer;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.SerializerUtil;
+import com.gemstone.gemfire.internal.logging.LogService;
 
 /**
  * A repository that writes to a single lucene index writer
@@ -48,6 +51,8 @@ public class IndexRepositoryImpl implements IndexRepository {
   private final SearcherManager searcherManager;
   private Region<?,?> region;
   
+  private static final Logger logger = LogService.getLogger();
+  
   public IndexRepositoryImpl(Region<?,?> region, IndexWriter writer, LuceneSerializer serializer) throws IOException {
     this.region = region;
     this.writer = writer;
@@ -85,6 +90,9 @@ public class IndexRepositoryImpl implements IndexRepository {
       for(ScoreDoc scoreDoc : docs.scoreDocs) {
         Document doc = searcher.doc(scoreDoc.doc);
         Object key = SerializerUtil.getKey(doc);
+        if (logger.isDebugEnabled()) {
+          logger.debug("query found doc:"+doc+":"+scoreDoc);
+        }
         collector.collect(key, scoreDoc.score);
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
index a0319f4..d2b1db1 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
@@ -20,8 +20,10 @@ package com.gemstone.gemfire.cache.lucene.internal.repository.serializer;
 
 import java.util.Map;
 
+import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
 
+import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.util.concurrent.CopyOnWriteWeakHashMap;
 import com.gemstone.gemfire.pdx.PdxInstance;
 
@@ -48,6 +50,8 @@ public class HeterogeneousLuceneSerializer implements LuceneSerializer {
    */
   private Map<Class<?>, LuceneSerializer> mappers = new CopyOnWriteWeakHashMap<Class<?>, LuceneSerializer>();
   
+  private static final Logger logger = LogService.getLogger();
+  
   public HeterogeneousLuceneSerializer(String[] indexedFields) {
     this.indexedFields = indexedFields;
     pdxMapper = new PdxLuceneSerializer(indexedFields);
@@ -59,6 +63,9 @@ public class HeterogeneousLuceneSerializer implements LuceneSerializer {
     LuceneSerializer mapper = getFieldMapper(value);
     
     mapper.toDocument(value, doc);
+    if (logger.isDebugEnabled()) {
+      logger.debug("HeterogeneousLuceneSerializer.toDocument:"+doc);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java
index c5c55a9..3990614 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java
@@ -19,8 +19,10 @@
 
 package com.gemstone.gemfire.cache.lucene.internal.repository.serializer;
 
+import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
 
+import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.pdx.PdxInstance;
 
 /**
@@ -30,6 +32,8 @@ class PdxLuceneSerializer implements LuceneSerializer {
 
   private String[] indexedFields;
 
+  private static final Logger logger = LogService.getLogger();
+
   public PdxLuceneSerializer(String[] indexedFields) {
     this.indexedFields = indexedFields;
   }
@@ -43,5 +47,8 @@ class PdxLuceneSerializer implements LuceneSerializer {
         SerializerUtil.addField(doc, field, fieldValue);
       }
     }
+    if (logger.isDebugEnabled()) {
+      logger.debug("PdxLuceneSerializer.toDocument:"+doc);
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java
index 953f31f..a76478c 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java
@@ -25,8 +25,11 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
 
+import com.gemstone.gemfire.internal.logging.LogService;
+
 /**
  * A lucene serializer that handles a single class and can
  * map an instance of that class to a document using reflection.
@@ -35,6 +38,8 @@ class ReflectionLuceneSerializer implements LuceneSerializer {
 
   private Field[] fields;
 
+  private static final Logger logger = LogService.getLogger();
+  
   public ReflectionLuceneSerializer(Class<? extends Object> clazz,
       String[] indexedFields) {
     Set<String> fieldSet = new HashSet<String>();
@@ -70,5 +75,8 @@ class ReflectionLuceneSerializer implements LuceneSerializer {
         //TODO - what to do if we can't read a field?
       }
     }
+    if (logger.isDebugEnabled()) {
+      logger.debug("ReflectionLuceneSerializer.toDocument:"+doc);
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
index 6d8e370..0d71cbb 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -61,6 +61,7 @@ public class LuceneIntegrationTest {
   protected CacheFactory getCacheFactory() {
     CacheFactory cf = new CacheFactory();
     cf.set("mcast-port", "0");
+    cf.set("log-level", System.getProperty("logLevel", "info"));
     cf.set("locators", "");
     return cf;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index 15f5747..4ebb9c4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -18,28 +18,31 @@ package com.gemstone.gemfire.cache.lucene;
 
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.verifyQueryKeys;
 import static org.hamcrest.Matchers.isA;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 import java.util.HashMap;
 import java.util.Map;
 
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.execute.FunctionException;
-import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
-import com.gemstone.gemfire.cache.lucene.test.TestObject;
-import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.analysis.core.LowerCaseFilter;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.analysis.util.CharTokenizer;
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
 /**
  * This class contains tests of lucene queries that can fit
  */
@@ -62,22 +65,77 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
 
     //Put two values with some of the same tokens
     String value1 = "one three";
-    region.put("A", new TestObject(value1, value1));
     String value2 = "one two three";
+    String value3 = "one@three";
+    region.put("A", new TestObject(value1, value1));
     region.put("B", new TestObject(value2, value2));
+    region.put("C", new TestObject(value3, value3));
 
+    // The value will be tokenized into following documents using the analyzers:
+    // <field1:one three> <field2:one three>
+    // <field1:one two three> <field2:one two three>
+    // <fi...@three> <fi...@three>
+    
     index.waitUntilFlushed(60000);
 
-    //Using the standard analyzer, this query will match both results
-    verifyQuery("field1:\"one three\"", "A", "B");
-
-    //Using the keyword analyzer, this query will only match the entry that exactly matches
+    // standard analyzer with double quote
+    // this query string will be parsed as "one three"
+    // but standard analyzer will parse value "one@three" to be "one three"
+    // query will be--fields1:"one three"
+    // so C will be hit by query
+    verifyQuery("field1:\"one three\"", "A", "C");
+    
+    // standard analyzer will not tokenize by '_'
+    // this query string will be parsed as "one_three"
+    // query will be--field1:one_three
+    verifyQuery("field1:one_three");
+    
+    // standard analyzer will tokenize by '@'
+    // this query string will be parsed as "one" "three"
+    // query will be--field1:one field1:three
+    verifyQuery("field1:one@three", "A", "B", "C");
+    
+    // keyword analyzer, this query will only match the entry that exactly matches
+    // this query string will be parsed as "one three"
+    // but keyword analyzer will parse one@three to be "one three"
+    // query will be--field2:one three
     verifyQuery("field2:\"one three\"", "A");
 
-
+    // keyword analyzer without double quote. It should be the same as 
+    // with double quote
+    // query will be--field2:one@three
+    verifyQuery("field2:one@three", "C");
   }
 
   @Test()
+  public void shouldTokenizeUsingMyCharacterAnalyser() throws ParseException {
+    Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
+    // not to specify field1's analyzer, it should use standard analyzer
+    // Note: fields has to contain "field1", otherwise, field1 will not be tokenized
+    fields.put("field1", null);
+    fields.put("field2", new MyCharacterAnalyzer());
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, fields);
+    Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
+      .create(REGION_NAME);
+    final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    //Put two values with some of the same tokens
+    String value1 = "one three";
+    String value4 = "two_four";
+    String value3 = "two@four";
+    region.put("A", new TestObject(value1, value4));
+    region.put("B", new TestObject(value1, value3));
+    region.put("C", new TestObject(value3, value3));
+    region.put("D", new TestObject(value4, value4));
+
+    index.waitUntilFlushed(60000);
+
+    verifyQuery("field1:one AND field2:two_four", "A");
+    verifyQuery("field1:one AND field2:two", "A");
+    verifyQuery("field1:three AND field2:four", "A");
+  }
+  
+  @Test()
   public void throwFunctionExceptionWhenGivenBadQuery() {
     LuceneService luceneService = LuceneServiceProvider.get(cache);
     luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
@@ -109,5 +167,20 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     verifyQueryKeys(queryWithStandardAnalyzer, expectedKeys);
   }
 
+  private static class MyCharacterTokenizer extends CharTokenizer {
+    @Override
+    protected boolean isTokenChar(final int character) {
+      return '_' != character;
+    }
+  }
+
+  private static class MyCharacterAnalyzer extends Analyzer {
+    @Override
+    protected TokenStreamComponents createComponents(final String field) {
+      Tokenizer tokenizer = new MyCharacterTokenizer();
+      TokenStream filter = new LowerCaseFilter(tokenizer);
+      return new TokenStreamComponents(tokenizer, filter);
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
index dded69c..cfd8c32 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
@@ -20,6 +20,8 @@ package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.assertEquals;
 
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.search.Query;
 import org.junit.Assert;
 import org.junit.Before;
@@ -36,12 +38,14 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class StringQueryProviderJUnitTest {
 
-  private LuceneIndex mockIndex;
+  private LuceneIndexImpl mockIndex;
 
   @Before
   public void initMocksAndCommonObjects() {
-    mockIndex = Mockito.mock(LuceneIndex.class, "mockIndex");
+    mockIndex = Mockito.mock(LuceneIndexImpl.class, "mockIndex");
     String[] fields = { "field-1", "field-2" };
+    Analyzer analyzer = new StandardAnalyzer();
+    Mockito.doReturn(analyzer).when(mockIndex).getAnalyzer();
     Mockito.doReturn(fields).when(mockIndex).getFieldNames();
     Mockito.doReturn("mockIndex").when(mockIndex).getName();
     Mockito.doReturn("mockRegionPath").when(mockIndex).getRegionPath();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
index 70ec434..c1a64ae 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
@@ -34,6 +34,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneIndex;
 import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneService;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexImpl;
 import com.gemstone.gemfire.cache.lucene.internal.StringQueryProvider;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
@@ -44,11 +45,14 @@ import com.gemstone.gemfire.internal.cache.InternalCache;
 import com.gemstone.gemfire.internal.cache.execute.InternalRegionFunctionContext;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.search.Query;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
 
 @Category(UnitTest.class)
 public class LuceneFunctionJUnitTest {
@@ -70,7 +74,7 @@ public class LuceneFunctionJUnitTest {
   IndexRepository mockRepository2;
   IndexResultCollector mockCollector;
   InternalLuceneService mockService;
-  InternalLuceneIndex mockIndex;
+  LuceneIndexImpl mockIndex;
 
   ArrayList<IndexRepository> repos;
   LuceneFunctionContext<IndexResultCollector> searchArgs;
@@ -263,10 +267,11 @@ public class LuceneFunctionJUnitTest {
     repos.add(mockRepository1);
     repos.add(mockRepository2);
     
-    mockIndex = mock(InternalLuceneIndex.class);
+    mockIndex = mock(LuceneIndexImpl.class);
     mockService = mock(InternalLuceneService.class);
     mockCache = mock(InternalCache.class);
-
+    Analyzer analyzer = new StandardAnalyzer();
+    Mockito.doReturn(analyzer).when(mockIndex).getAnalyzer();
     queryProvider = new StringQueryProvider("gemfire:lucene");
     
     searchArgs = new LuceneFunctionContext<IndexResultCollector>(queryProvider, "indexName");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/777c42ee/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index 571049c..860dacf 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -75,7 +75,7 @@ public class LuceneTestUtilities {
    */
   public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) {
     Set<K> expectedKeySet = new HashSet<>(Arrays.asList(expectedKeys));
-    Set<K> actualKeySet = new HashSet<>(Arrays.asList(expectedKeys));
+    Set<K> actualKeySet = new HashSet<>();
     final LuceneQueryResults<K, Object> results = query.search();
     while(results.hasNextPage()) {
       results.getNextPage().stream()